diff --git a/.github/workflows/compile_br.yaml b/.github/workflows/compile_br.yaml index da5ef43198ffc..acfbc2d27bad5 100644 --- a/.github/workflows/compile_br.yaml +++ b/.github/workflows/compile_br.yaml @@ -13,7 +13,10 @@ on: - '!br/docs/**' - '!br/tests/**' - '!br/docker/**' +#change trigger policy pull_request: + types: + - labeled # <-- branches: - master - 'release-[0-9].[0-9]*' @@ -35,8 +38,25 @@ concurrency: cancel-in-progress: true jobs: + compile-windows: + if: github.event_name == 'push' || github.event_name == 'pull_request' && github.event.label.name == 'action/run-br-cross-platform-build' + name: Compile for Windows job + runs-on: windows-latest + steps: + - uses: actions/checkout@v2.1.0 + + - name: Set up Go + uses: actions/setup-go@v2 + with: + go-version: 1.16 + + - name: Run build + run: make build_tools + compile: + if: github.event_name == 'pull_request' && github.event.label.name == 'action/run-br-cross-platform-build' name: Compile for ${{ matrix.os }} / ${{ matrix.target}} + runs-on: ${{ matrix.os }} strategy: matrix: @@ -47,8 +67,6 @@ jobs: - os: ubuntu-latest target: aarch64-unknown-linux-gnu - - os: windows-latest - target: x86_64-pc-windows-msvc steps: - uses: actions/checkout@v2.1.0 @@ -61,6 +79,7 @@ jobs: run: make build_tools compile-freebsd: + if: github.event_name == 'pull_request' && github.event.label.name == 'action/run-br-cross-platform-build' name: Compile for FreeBSD job runs-on: ubuntu-latest steps: diff --git a/Makefile b/Makefile index d75d8d0a556e3..0e5e9879816fd 100644 --- a/Makefile +++ b/Makefile @@ -109,11 +109,11 @@ explaintest: server_check ddltest: @cd cmd/ddltest && $(GO) test -o ../../bin/ddltest -c -upload-coverage: SHELL:=/bin/bash upload-coverage: -ifeq ("$(TRAVIS_COVERAGE)", "1") - mv overalls.coverprofile coverage.txt - bash <(curl -s https://codecov.io/bash) +ifneq ($(CODECOV_TOKEN), "") + curl -LO ${FILE_SERVER_URL}/download/cicd/ci-tools/codecov + chmod +x codecov + ./codecov -t ${CODECOV_TOKEN} endif devgotest: failpoint-enable @@ -129,7 +129,7 @@ devgotest: failpoint-enable gotest: failpoint-enable @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); cat 'gotest.log'; exit 1; } + $(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -cover $(PACKAGES_TIDB_TESTS) -coverprofile=coverage.txt -check.p true > gotest.log || { $(FAILPOINT_DISABLE); cat 'gotest.log'; exit 1; } @$(FAILPOINT_DISABLE) race: failpoint-enable diff --git a/bindinfo/capture_serial_test.go b/bindinfo/capture_serial_test.go index bb73a3777a09d..e65697c3f2a21 100644 --- a/bindinfo/capture_serial_test.go +++ b/bindinfo/capture_serial_test.go @@ -332,9 +332,8 @@ func TestCapturedBindingCharset(t *testing.T) { require.Len(t, rows, 1) require.Equal(t, "update `test` . `t` set `name` = ? where `name` <= ?", rows[0][0]) require.Equal(t, "UPDATE /*+ use_index(@`upd_1` `test`.`t` `idx`)*/ `test`.`t` SET `name`='hello' WHERE `name` <= 'abc'", rows[0][1]) - // Charset and Collation are empty now, they are not used currently. - require.Equal(t, "", rows[0][6]) - require.Equal(t, "", rows[0][7]) + require.Equal(t, "utf8mb4", rows[0][6]) + require.Equal(t, "utf8mb4_bin", rows[0][7]) } func TestConcurrentCapture(t *testing.T) { diff --git a/br/pkg/backup/schema.go b/br/pkg/backup/schema.go index b1b0e8e16f0f6..4e653253cafcd 100644 --- a/br/pkg/backup/schema.go +++ b/br/pkg/backup/schema.go @@ -88,10 +88,13 @@ func (ss *Schemas) BackupSchemas( metaWriter.StartWriteMetasAsync(ctx, op) for _, s := range ss.schemas { schema := s + // Because schema.dbInfo is a pointer that many tables point to. + // Remove "add Temporary-prefix into dbName" from closure to prevent concurrent operations. + if utils.IsSysDB(schema.dbInfo.Name.L) { + schema.dbInfo.Name = utils.TemporaryDBName(schema.dbInfo.Name.O) + } + workerPool.ApplyOnErrorGroup(errg, func() error { - if utils.IsSysDB(schema.dbInfo.Name.L) { - schema.dbInfo.Name = utils.TemporaryDBName(schema.dbInfo.Name.O) - } logger := log.With( zap.String("db", schema.dbInfo.Name.O), zap.String("table", schema.tableInfo.Name.O), diff --git a/br/pkg/backup/schema_test.go b/br/pkg/backup/schema_test.go index 16384d495b285..c858d556f98d0 100644 --- a/br/pkg/backup/schema_test.go +++ b/br/pkg/backup/schema_test.go @@ -4,7 +4,9 @@ package backup_test import ( "context" + "fmt" "math" + "strings" "sync/atomic" "github.com/golang/protobuf/proto" @@ -16,6 +18,7 @@ import ( "github.com/pingcap/tidb/br/pkg/metautil" "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -260,3 +263,42 @@ func (s *testBackupSchemaSuite) TestBuildBackupRangeAndSchemaWithBrokenStats(c * c.Assert(schemas2[0].Info, DeepEquals, schemas[0].Info) c.Assert(schemas2[0].DB, DeepEquals, schemas[0].DB) } + +func (s *testBackupSchemaSuite) TestBackupSchemasForSystemTable(c *C) { + tk := testkit.NewTestKit(c, s.mock.Storage) + es2 := s.GetRandomStorage(c) + + systemTablesCount := 32 + tablePrefix := "systable" + tk.MustExec("use mysql") + for i := 1; i <= systemTablesCount; i++ { + query := fmt.Sprintf("create table %s%d (a char(1));", tablePrefix, i) + tk.MustExec(query) + } + + f, err := filter.Parse([]string{"mysql.systable*"}) + c.Assert(err, IsNil) + _, backupSchemas, err := backup.BuildBackupRangeAndSchema(s.mock.Storage, f, math.MaxUint64) + c.Assert(err, IsNil) + c.Assert(backupSchemas.Len(), Equals, systemTablesCount) + + ctx := context.Background() + cipher := backuppb.CipherInfo{ + CipherType: encryptionpb.EncryptionMethod_PLAINTEXT, + } + updateCh := new(simpleProgress) + + metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, &cipher) + err = backupSchemas.BackupSchemas(ctx, metaWriter2, s.mock.Storage, nil, + math.MaxUint64, 1, variable.DefChecksumTableConcurrency, true, updateCh) + c.Assert(err, IsNil) + err = metaWriter2.FlushBackupMeta(ctx) + c.Assert(err, IsNil) + + schemas2 := s.GetSchemasFromMeta(c, es2) + c.Assert(schemas2, HasLen, systemTablesCount) + for _, schema := range schemas2 { + c.Assert(schema.DB.Name, Equals, utils.TemporaryDBName("mysql")) + c.Assert(strings.HasPrefix(schema.Info.Name.O, tablePrefix), Equals, true) + } +} diff --git a/br/pkg/lightning/backend/tidb/tidb_test.go b/br/pkg/lightning/backend/tidb/tidb_test.go index 3739bf7551db9..e37f53d12b27e 100644 --- a/br/pkg/lightning/backend/tidb/tidb_test.go +++ b/br/pkg/lightning/backend/tidb/tidb_test.go @@ -417,13 +417,10 @@ func (s *mysqlSuite) TestWriteRowsErrorDowngrading(c *C) { ExpectExec("INSERT INTO `tidb_lightning_errors`\\.type_error_v1.*"). WithArgs(sqlmock.AnyArg(), "`foo`.`bar`", "9.csv", int64(0), nonRetryableError.Error(), "(3)"). WillReturnResult(driver.ResultNoRows) + // the forth row will exceed the error threshold, won't record this error s.mockDB. ExpectExec("\\QINSERT INTO `foo`.`bar`(`a`) VALUES(4)\\E"). WillReturnError(nonRetryableError) - s.mockDB. - ExpectExec("INSERT INTO `tidb_lightning_errors`\\.type_error_v1.*"). - WithArgs(sqlmock.AnyArg(), "`foo`.`bar`", "10.csv", int64(0), nonRetryableError.Error(), "(4)"). - WillReturnResult(driver.ResultNoRows) ctx := context.Background() logger := log.L() diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index b0c926d561691..8f2e6f2dfa9ac 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -585,6 +585,48 @@ func (d *Duration) MarshalJSON() ([]byte, error) { return []byte(fmt.Sprintf(`"%s"`, d.Duration)), nil } +// Charset defines character set +type Charset int + +const ( + Binary Charset = iota + UTF8MB4 + GB18030 + GBK +) + +// String return the string value of charset +func (c Charset) String() string { + switch c { + case Binary: + return "binary" + case UTF8MB4: + return "utf8mb4" + case GB18030: + return "gb18030" + case GBK: + return "gbk" + default: + return "unknown_charset" + } +} + +// ParseCharset parser character set for string +func ParseCharset(dataCharacterSet string) (Charset, error) { + switch strings.ToLower(dataCharacterSet) { + case "", "binary": + return Binary, nil + case "utf8mb4": + return UTF8MB4, nil + case "gb18030": + return GB18030, nil + case "gbk": + return GBK, nil + default: + return Binary, errors.Errorf("found unsupported data-character-set: %s", dataCharacterSet) + } +} + func NewConfig() *Config { return &Config{ App: Lightning{ @@ -786,6 +828,16 @@ func (cfg *Config) Adjust(ctx context.Context) error { if len(cfg.Mydumper.DataCharacterSet) == 0 { cfg.Mydumper.DataCharacterSet = defaultCSVDataCharacterSet } + charset, err1 := ParseCharset(cfg.Mydumper.DataCharacterSet) + if err1 != nil { + return err1 + } + if charset == GBK || charset == GB18030 { + log.L().Warn( + "incompatible strings may be encountered during the transcoding process and will be replaced, please be aware of the risk of not being able to retain the original information", + zap.String("source-character-set", charset.String()), + zap.ByteString("invalid-char-replacement", []byte(cfg.Mydumper.DataInvalidCharReplace))) + } if cfg.TikvImporter.Backend == "" { return errors.New("tikv-importer.backend must not be empty!") diff --git a/br/pkg/lightning/config/config_test.go b/br/pkg/lightning/config/config_test.go index 8fb5f2f61ea41..1e7e751b20b3d 100644 --- a/br/pkg/lightning/config/config_test.go +++ b/br/pkg/lightning/config/config_test.go @@ -854,3 +854,23 @@ func (s *configTestSuite) TestCheckpointKeepStrategy(c *C) { c.Assert(res, DeepEquals, []byte(value)) } } + +func (s configTestSuite) TestLoadCharsetFromConfig(c *C) { + cases := map[string]config.Charset{ + "binary": config.Binary, + "BINARY": config.Binary, + "GBK": config.GBK, + "gbk": config.GBK, + "Gbk": config.GBK, + "gB18030": config.GB18030, + "GB18030": config.GB18030, + } + for k, v := range cases { + charset, err := config.ParseCharset(k) + c.Assert(err, IsNil) + c.Assert(charset, Equals, v) + } + + _, err := config.ParseCharset("Unknown") + c.Assert(err, ErrorMatches, "found unsupported data-character-set: Unknown") +} diff --git a/br/pkg/lightning/errormanager/errormanager.go b/br/pkg/lightning/errormanager/errormanager.go index aaa155576bb19..e301af21cb826 100644 --- a/br/pkg/lightning/errormanager/errormanager.go +++ b/br/pkg/lightning/errormanager/errormanager.go @@ -1,9 +1,24 @@ +// 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 errormanager import ( "context" "database/sql" "fmt" + "strings" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/common" @@ -94,6 +109,7 @@ type ErrorManager struct { taskID int64 schemaEscaped string remainingError config.MaxError + dupResolution config.DuplicateResolutionAlgorithm } // New creates a new error manager. @@ -111,7 +127,7 @@ func New(db *sql.DB, cfg *config.Config) *ErrorManager { // Init creates the schemas and tables to store the task information. func (em *ErrorManager) Init(ctx context.Context) error { - if em.db == nil { + if em.db == nil || (em.remainingError.Type.Load() == 0 && em.dupResolution == config.DupeResAlgNone) { return nil } @@ -120,15 +136,21 @@ func (em *ErrorManager) Init(ctx context.Context) error { Logger: log.L(), } - sqls := [][2]string{ - {"create task info schema", createSchema}, - {"create syntax error table", createSyntaxErrorTable}, - {"create type error table", createTypeErrorTable}, - {"create conflict error table", createConflictErrorTable}, + sqls := make([][2]string, 0) + sqls = append(sqls, [2]string{"create task info schema", createSchema}) + if em.remainingError.Syntax.Load() > 0 { + sqls = append(sqls, [2]string{"create syntax error table", createSyntaxErrorTable}) + } + if em.remainingError.Type.Load() > 0 { + sqls = append(sqls, [2]string{"create type error table", createTypeErrorTable}) + } + if em.dupResolution != config.DupeResAlgNone && em.remainingError.Conflict.Load() > 0 { + sqls = append(sqls, [2]string{"create conflict error table", createConflictErrorTable}) } for _, sql := range sqls { - err := exec.Exec(ctx, sql[0], fmt.Sprintf(sql[1], em.schemaEscaped)) + // trim spaces for unit test pattern matching + err := exec.Exec(ctx, sql[0], strings.TrimSpace(fmt.Sprintf(sql[1], em.schemaEscaped))) if err != nil { return err } @@ -148,6 +170,11 @@ func (em *ErrorManager) RecordTypeError( rowText string, encodeErr error, ) error { + // elide the encode error if needed. + if em.remainingError.Type.Dec() < 0 { + return encodeErr + } + if em.db != nil { errMsg := encodeErr.Error() logger = logger.With( @@ -173,11 +200,6 @@ func (em *ErrorManager) RecordTypeError( return multierr.Append(encodeErr, err) } } - - // elide the encode error if needed. - if em.remainingError.Type.Dec() < 0 { - return encodeErr - } return nil } diff --git a/br/pkg/lightning/errormanager/errormanager_test.go b/br/pkg/lightning/errormanager/errormanager_test.go new file mode 100644 index 0000000000000..4434cff5012bc --- /dev/null +++ b/br/pkg/lightning/errormanager/errormanager_test.go @@ -0,0 +1,83 @@ +// 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 errormanager + +import ( + "context" + "math" + "testing" + + "github.com/DATA-DOG/go-sqlmock" + . "github.com/pingcap/check" + "go.uber.org/atomic" + + "github.com/pingcap/tidb/br/pkg/lightning/config" +) + +var _ = Suite(errorManagerSuite{}) + +func TestErrorManager(t *testing.T) { + TestingT(t) +} + +type errorManagerSuite struct{} + +func (e errorManagerSuite) TestInit(c *C) { + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + + em := &ErrorManager{ + db: db, + schemaEscaped: "`lightning_errors`", + remainingError: config.MaxError{ + Charset: *atomic.NewInt64(math.MaxInt64), + Conflict: *atomic.NewInt64(math.MaxInt64), + }, + } + + ctx := context.Background() + err = em.Init(ctx) + c.Assert(err, IsNil) + + em.dupResolution = config.DupeResAlgRecord + mock.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_errors`;"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.conflict_error_v1.*"). + WillReturnResult(sqlmock.NewResult(2, 1)) + err = em.Init(ctx) + c.Assert(err, IsNil) + + em.dupResolution = config.DupeResAlgNone + em.remainingError.Type.Store(1) + mock.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_errors`;"). + WillReturnResult(sqlmock.NewResult(3, 1)) + mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.type_error_v1.*"). + WillReturnResult(sqlmock.NewResult(4, 1)) + err = em.Init(ctx) + c.Assert(err, IsNil) + + em.dupResolution = config.DupeResAlgRecord + em.remainingError.Type.Store(1) + mock.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_errors`.*"). + WillReturnResult(sqlmock.NewResult(5, 1)) + mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.type_error_v1.*"). + WillReturnResult(sqlmock.NewResult(6, 1)) + mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.conflict_error_v1.*"). + WillReturnResult(sqlmock.NewResult(7, 1)) + err = em.Init(ctx) + c.Assert(err, IsNil) + + c.Assert(mock.ExpectationsWereMet(), IsNil) +} diff --git a/br/pkg/lightning/mydump/charset_convertor.go b/br/pkg/lightning/mydump/charset_convertor.go index e4cd7a4e72c1d..81e57be681a18 100644 --- a/br/pkg/lightning/mydump/charset_convertor.go +++ b/br/pkg/lightning/mydump/charset_convertor.go @@ -19,41 +19,17 @@ import ( "unicode/utf8" "github.com/pingcap/errors" - "github.com/pingcap/tidb/br/pkg/lightning/log" - "go.uber.org/zap" "golang.org/x/text/encoding" "golang.org/x/text/encoding/simplifiedchinese" -) - -type Charset int -const ( - Binary Charset = iota - UTF8MB4 - GB18030 - GBK + "github.com/pingcap/tidb/br/pkg/lightning/config" ) -func (c Charset) String() string { - switch c { - case Binary: - return "binary" - case UTF8MB4: - return "utf8mb4" - case GB18030: - return "gb18030" - case GBK: - return "gbk" - default: - return "unknown_charset" - } -} - // CharsetConvertor is used to convert a character set to utf8mb4 encoding. // In Lightning, we mainly use it to do the GB18030/GBK -> UTF8MB4 conversion. type CharsetConvertor struct { // sourceCharacterSet represents the charset that the data source uses. - sourceCharacterSet Charset + sourceCharacterSet config.Charset // invalidCharReplacement is the default replacement character bytes for the invalid content, e.g "\ufffd". invalidCharReplacement string @@ -63,14 +39,10 @@ type CharsetConvertor struct { // NewCharsetConvertor creates a new CharsetConvertor. func NewCharsetConvertor(dataCharacterSet, dataInvalidCharReplace string) (*CharsetConvertor, error) { - sourceCharacterSet, err := loadCharsetFromConfig(dataCharacterSet) + sourceCharacterSet, err := config.ParseCharset(dataCharacterSet) if err != nil { return nil, err } - log.L().Warn( - "incompatible strings may be encountered during the transcoding process and will be replaced, please be aware of the risk of not being able to retain the original information", - zap.String("source-character-set", sourceCharacterSet.String()), - zap.ByteString("invalid-char-replacement", []byte(dataInvalidCharReplace))) cc := &CharsetConvertor{ sourceCharacterSet, dataInvalidCharReplace, @@ -87,29 +59,14 @@ func NewCharsetConvertor(dataCharacterSet, dataInvalidCharReplace string) (*Char return cc, nil } -func loadCharsetFromConfig(dataCharacterSet string) (Charset, error) { - switch dataCharacterSet { - case "", "binary": - return Binary, nil - case "utf8mb4": - return UTF8MB4, nil - case "gb18030": - return GB18030, nil - case "gbk": - return GBK, nil - default: - return Binary, errors.Errorf("found unsupported data-character-set: %s", dataCharacterSet) - } -} - func (cc *CharsetConvertor) initDecoder() error { switch cc.sourceCharacterSet { - case Binary, UTF8MB4: + case config.Binary, config.UTF8MB4: return nil - case GB18030: + case config.GB18030: cc.decoder = simplifiedchinese.GB18030.NewDecoder() return nil - case GBK: + case config.GBK: cc.decoder = simplifiedchinese.GBK.NewDecoder() return nil } @@ -118,12 +75,12 @@ func (cc *CharsetConvertor) initDecoder() error { func (cc *CharsetConvertor) initEncoder() error { switch cc.sourceCharacterSet { - case Binary, UTF8MB4: + case config.Binary, config.UTF8MB4: return nil - case GB18030: + case config.GB18030: cc.encoder = simplifiedchinese.GB18030.NewEncoder() return nil - case GBK: + case config.GBK: cc.encoder = simplifiedchinese.GBK.NewEncoder() return nil } @@ -151,7 +108,7 @@ func (cc *CharsetConvertor) Decode(src string) (string, error) { func (cc *CharsetConvertor) precheck(src string) bool { // No need to convert the charset encoding, just return the original data. if len(src) == 0 || cc == nil || - cc.sourceCharacterSet == Binary || cc.sourceCharacterSet == UTF8MB4 || + cc.sourceCharacterSet == config.Binary || cc.sourceCharacterSet == config.UTF8MB4 || cc.decoder == nil || cc.encoder == nil { return false } diff --git a/br/pkg/lightning/mydump/charset_convertor_test.go b/br/pkg/lightning/mydump/charset_convertor_test.go index 5220f0575360c..cf091c09b142e 100644 --- a/br/pkg/lightning/mydump/charset_convertor_test.go +++ b/br/pkg/lightning/mydump/charset_convertor_test.go @@ -12,14 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mydump_test +package mydump import ( "io" "os" . "github.com/pingcap/check" - "github.com/pingcap/tidb/br/pkg/lightning/mydump" ) var _ = Suite(&testCharsetConvertorSuite{}) @@ -51,7 +50,7 @@ func (s testCharsetConvertorSuite) TestCharsetConvertor(c *C) { gbkData, err := io.ReadAll(gbkReader) c.Assert(err, IsNil) - cc, err := mydump.NewCharsetConvertor("gb18030", "\ufffd") + cc, err := NewCharsetConvertor("gb18030", "\ufffd") c.Assert(err, IsNil) gbkToUTF8Data, err := cc.Decode(string(gbkData)) c.Assert(err, IsNil) @@ -79,7 +78,7 @@ func (s testCharsetConvertorSuite) TestInvalidCharReplace(c *C) { c.Assert(err, IsNil) gbkData, err := io.ReadAll(gbkReader) c.Assert(err, IsNil) - cc, err := mydump.NewCharsetConvertor("gb18030", dataInvalidCharReplace) + cc, err := NewCharsetConvertor("gb18030", dataInvalidCharReplace) c.Assert(err, IsNil) gbkToUTF8Data, err := cc.Decode(string(gbkData)) c.Assert(err, IsNil) diff --git a/br/pkg/lightning/restore/checksum_test.go b/br/pkg/lightning/restore/checksum_test.go index 0882c172f6af6..6a9f334b31f9a 100644 --- a/br/pkg/lightning/restore/checksum_test.go +++ b/br/pkg/lightning/restore/checksum_test.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/memory" tmock "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/trxevents" @@ -96,11 +97,9 @@ func (s *checksumSuite) TestDoChecksumParallel(c *C) { // db.Close() will close all connections from its idle pool, set it 1 to expect one close db.SetMaxIdleConns(1) - var wg sync.WaitGroup - wg.Add(5) + var wg util.WaitGroupWrapper for i := 0; i < 5; i++ { - go func() { - defer wg.Done() + wg.Run(func() { checksum, err := DoChecksum(ctx, &TidbTableInfo{DB: "test", Name: "t"}) c.Assert(err, IsNil) c.Assert(*checksum, DeepEquals, RemoteChecksum{ @@ -110,7 +109,7 @@ func (s *checksumSuite) TestDoChecksumParallel(c *C) { TotalKVs: 7296873, TotalBytes: 357601387, }) - }() + }) } wg.Wait() @@ -136,14 +135,13 @@ func (s *checksumSuite) TestIncreaseGCLifeTimeFail(c *C) { mock.ExpectClose() ctx := MockDoChecksumCtx(db) - var wg sync.WaitGroup - wg.Add(5) + var wg util.WaitGroupWrapper + for i := 0; i < 5; i++ { - go func() { + wg.Run(func() { _, errChecksum := DoChecksum(ctx, &TidbTableInfo{DB: "test", Name: "t"}) c.Assert(errChecksum, ErrorMatches, "update GC lifetime failed: update gc error: context canceled") - wg.Done() - }() + }) } wg.Wait() diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 43416d8157099..8da5a210ce885 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -796,7 +796,8 @@ func (tr *TableRestore) postProcess( nextStage = checkpoints.CheckpointStatusChecksumSkipped } - if err == nil { + // Don't call FinishTable when other lightning will calculate checksum. + if err == nil && !hasDupe && needChecksum { err = metaMgr.FinishTable(ctx) } diff --git a/br/pkg/mock/mock_cluster.go b/br/pkg/mock/mock_cluster.go index 24e78877939d7..c60f4764b4ffc 100644 --- a/br/pkg/mock/mock_cluster.go +++ b/br/pkg/mock/mock_cluster.go @@ -108,6 +108,7 @@ func (mock *Cluster) Start() error { cfg.Store = "tikv" cfg.Status.StatusPort = uint(statusPort) cfg.Status.ReportStatus = true + cfg.Socket = fmt.Sprintf("/tmp/tidb-mock-%d.sock", time.Now().UnixNano()) svr, err := server.NewServer(cfg, mock.TiDBDriver) if err != nil { diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index 45a88893273be..c62eb1c3af6d0 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -810,6 +810,7 @@ func (rc *Client) GoValidateChecksum( close(loadStatCh) wg.Done() }() + for { select { // if we use ectx here, maybe canceled will mask real error. @@ -819,11 +820,11 @@ func (rc *Client) GoValidateChecksum( if !ok { return } + workers.ApplyOnErrorGroup(eg, func() error { start := time.Now() defer func() { elapsed := time.Since(start) - summary.CollectDuration("restore checksum", elapsed) summary.CollectSuccessUnit("table checksum", 1, elapsed) }() err := rc.execChecksum(ectx, tbl, kvClient, concurrency, loadStatCh) diff --git a/br/pkg/restore/systable_restore.go b/br/pkg/restore/systable_restore.go index 1f7d6bb12ca9c..f6235d195a850 100644 --- a/br/pkg/restore/systable_restore.go +++ b/br/pkg/restore/systable_restore.go @@ -34,15 +34,16 @@ var unRecoverableTable = map[string]struct{}{ "global_variables": {}, // all user related tables cannot be recovered for now. - "columns_priv": {}, - "db": {}, - "default_roles": {}, - "global_grants": {}, - "global_priv": {}, - "role_edges": {}, - "tables_priv": {}, - "user": {}, - + "column_stats_usage": {}, + "columns_priv": {}, + "db": {}, + "default_roles": {}, + "global_grants": {}, + "global_priv": {}, + "role_edges": {}, + "tables_priv": {}, + "user": {}, + "capture_plan_baselines_blacklist": {}, // gc info don't need to recover. "gc_delete_range": {}, "gc_delete_range_done": {}, diff --git a/br/pkg/version/build/info_test.go b/br/pkg/version/build/info_test.go index b90deab036407..4714d21e78232 100644 --- a/br/pkg/version/build/info_test.go +++ b/br/pkg/version/build/info_test.go @@ -6,28 +6,22 @@ import ( "strings" "testing" - . "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -type infoSuite struct{} +func TestInfo(t *testing.T) { + t.Parallel() -var _ = Suite(&infoSuite{}) - -func TestT(t *testing.T) { - TestingT(t) -} - -func (*infoSuite) TestInfo(c *C) { info := Info() lines := strings.Split(info, "\n") - c.Assert(lines[0], Matches, "Release Version.*") - c.Assert(lines[1], Matches, "Git Commit Hash.*") - c.Assert(lines[2], Matches, "Git Branch.*") - c.Assert(lines[3], Matches, "Go Version.*") - c.Assert(lines[4], Matches, "UTC Build Time.*") + require.Regexp(t, "Release Version.*", lines[0]) + require.Regexp(t, "Git Commit Hash.*", lines[1]) + require.Regexp(t, "Git Branch.*", lines[2]) + require.Regexp(t, "Go Version.*", lines[3]) + require.Regexp(t, "UTC Build Time.*", lines[4]) } -func (*infoSuite) TestLogInfo(c *C) { +func TestLogInfo(t *testing.T) { LogInfo(BR) LogInfo(Lightning) } diff --git a/br/pkg/version/version_test.go b/br/pkg/version/version_test.go index e1f614c313699..a30668b6f99d6 100644 --- a/br/pkg/version/version_test.go +++ b/br/pkg/version/version_test.go @@ -11,20 +11,12 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/coreos/go-semver/semver" - . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/br/pkg/version/build" + "github.com/stretchr/testify/require" pd "github.com/tikv/pd/client" ) -type checkSuite struct{} - -var _ = Suite(&checkSuite{}) - -func TestT(t *testing.T) { - TestingT(t) -} - type mockPDClient struct { pd.Client getAllStores func() []*metapb.Store @@ -43,7 +35,9 @@ func tiflash(version string) []*metapb.Store { } } -func (s *checkSuite) TestCheckClusterVersion(c *C) { +func TestCheckClusterVersion(t *testing.T) { + t.Parallel() + mock := mockPDClient{ Client: nil, } @@ -54,7 +48,8 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return tiflash("v4.0.0-rc.1") } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, ErrorMatches, `incompatible.*version v4.0.0-rc.1, try update it to 4.0.0.*`) + require.Error(t, err) + require.Regexp(t, `incompatible.*version v4.0.0-rc.1, try update it to 4.0.0.*`, err.Error()) } { @@ -63,7 +58,8 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return tiflash("v3.1.0-beta.1") } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, ErrorMatches, `incompatible.*version v3.1.0-beta.1, try update it to 3.1.0.*`) + require.Error(t, err) + require.Regexp(t, `incompatible.*version v3.1.0-beta.1, try update it to 3.1.0.*`, err.Error()) } { @@ -72,7 +68,8 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return tiflash("v3.0.15") } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, ErrorMatches, `incompatible.*version v3.0.15, try update it to 3.1.0.*`) + require.Error(t, err) + require.Regexp(t, `incompatible.*version v3.0.15, try update it to 3.1.0.*`, err.Error()) } { @@ -81,7 +78,7 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: minTiKVVersion.String()}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, IsNil) + require.NoError(t, err) } { @@ -91,7 +88,8 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: `v2.1.0`}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, ErrorMatches, ".*TiKV .* don't support BR, please upgrade cluster .*") + require.Error(t, err) + require.Regexp(t, ".*TiKV .* don't support BR, please upgrade cluster .*", err.Error()) } { @@ -101,7 +99,8 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: minTiKVVersion.String()}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, ErrorMatches, "TiKV .* mismatch, please .*") + require.Error(t, err) + require.Regexp(t, "TiKV .* mismatch, please .*", err.Error()) } { @@ -111,7 +110,8 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: "v4.0.0-rc"}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, ErrorMatches, "TiKV .* major version mismatch, please .*") + require.Error(t, err) + require.Regexp(t, "TiKV .* major version mismatch, please .*", err.Error()) } { @@ -121,7 +121,8 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: "v4.0.0-beta.1"}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, ErrorMatches, "TiKV .* mismatch, please .*") + require.Error(t, err) + require.Regexp(t, "TiKV .* mismatch, please .*", err.Error()) } { @@ -131,7 +132,7 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: "v4.0.0-rc.1"}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, IsNil) + require.NoError(t, err) } { @@ -140,7 +141,7 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: "v4.0.0-rc.1"}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBackup(semver.New("4.0.12"))) - c.Assert(err, IsNil) + require.NoError(t, err) } { @@ -149,7 +150,7 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: "v4.0.0-rc.1"}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBackup(semver.New("5.0.0-rc"))) - c.Assert(err, Not(IsNil)) + require.Error(t, err) } { @@ -159,127 +160,123 @@ func (s *checkSuite) TestCheckClusterVersion(c *C) { return []*metapb.Store{{Version: "v4.0.0-rc.2"}} } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) - c.Assert(err, IsNil) + require.NoError(t, err) } } -func (s *checkSuite) TestCompareVersion(c *C) { - c.Assert(semver.New("4.0.0-rc").Compare(*semver.New("4.0.0-rc.2")), Equals, -1) - c.Assert(semver.New("4.0.0-beta.3").Compare(*semver.New("4.0.0-rc.2")), Equals, -1) - c.Assert(semver.New("4.0.0-rc.1").Compare(*semver.New("4.0.0")), Equals, -1) - c.Assert(semver.New("4.0.0-beta.1").Compare(*semver.New("4.0.0")), Equals, -1) - c.Assert(semver.New(removeVAndHash("4.0.0-rc-35-g31dae220")).Compare(*semver.New("4.0.0-rc.2")), Equals, -1) - c.Assert(semver.New(removeVAndHash("4.0.0-9-g30f0b014")).Compare(*semver.New("4.0.0-rc.1")), Equals, 1) - c.Assert(semver.New(removeVAndHash("v3.0.0-beta-211-g09beefbe0-dirty")). - Compare(*semver.New("3.0.0-beta")), Equals, 0) - c.Assert(semver.New(removeVAndHash("v3.0.5-dirty")). - Compare(*semver.New("3.0.5")), Equals, 0) - c.Assert(semver.New(removeVAndHash("v3.0.5-beta.12-dirty")). - Compare(*semver.New("3.0.5-beta.12")), Equals, 0) - c.Assert(semver.New(removeVAndHash("v2.1.0-rc.1-7-g38c939f-dirty")). - Compare(*semver.New("2.1.0-rc.1")), Equals, 0) +func TestCompareVersion(t *testing.T) { + t.Parallel() + + require.Equal(t, -1, semver.New("4.0.0-rc").Compare(*semver.New("4.0.0-rc.2"))) + require.Equal(t, -1, semver.New("4.0.0-beta.3").Compare(*semver.New("4.0.0-rc.2"))) + require.Equal(t, -1, semver.New("4.0.0-rc.1").Compare(*semver.New("4.0.0"))) + require.Equal(t, -1, semver.New("4.0.0-beta.1").Compare(*semver.New("4.0.0"))) + require.Equal(t, -1, semver.New(removeVAndHash("4.0.0-rc-35-g31dae220")).Compare(*semver.New("4.0.0-rc.2"))) + require.Equal(t, 1, semver.New(removeVAndHash("4.0.0-9-g30f0b014")).Compare(*semver.New("4.0.0-rc.1"))) + require.Equal(t, 0, semver.New(removeVAndHash("v3.0.0-beta-211-g09beefbe0-dirty")). + Compare(*semver.New("3.0.0-beta"))) + require.Equal(t, 0, semver.New(removeVAndHash("v3.0.5-dirty")). + Compare(*semver.New("3.0.5"))) + require.Equal(t, 0, semver.New(removeVAndHash("v3.0.5-beta.12-dirty")). + Compare(*semver.New("3.0.5-beta.12"))) + require.Equal(t, 0, semver.New(removeVAndHash("v2.1.0-rc.1-7-g38c939f-dirty")). + Compare(*semver.New("2.1.0-rc.1"))) } -func (s *checkSuite) TestNextMajorVersion(c *C) { +func TestNextMajorVersion(t *testing.T) { + t.Parallel() + build.ReleaseVersion = "v4.0.0-rc.1" - c.Assert(NextMajorVersion().String(), Equals, "5.0.0") + require.Equal(t, "5.0.0", NextMajorVersion().String()) build.ReleaseVersion = "4.0.0-rc-35-g31dae220" - c.Assert(NextMajorVersion().String(), Equals, "5.0.0") + require.Equal(t, "5.0.0", NextMajorVersion().String()) build.ReleaseVersion = "4.0.0-9-g30f0b014" - c.Assert(NextMajorVersion().String(), Equals, "5.0.0") + require.Equal(t, "5.0.0", NextMajorVersion().String()) build.ReleaseVersion = "v5.0.0-rc.2" - c.Assert(NextMajorVersion().String(), Equals, "6.0.0") + require.Equal(t, "6.0.0", NextMajorVersion().String()) build.ReleaseVersion = "v5.0.0-master" - c.Assert(NextMajorVersion().String(), Equals, "6.0.0") + require.Equal(t, "6.0.0", NextMajorVersion().String()) } -func (s *checkSuite) TestExtractTiDBVersion(c *C) { +func TestExtractTiDBVersion(t *testing.T) { + t.Parallel() + vers, err := ExtractTiDBVersion("5.7.10-TiDB-v2.1.0-rc.1-7-g38c939f") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("2.1.0-rc.1")) + require.NoError(t, err) + require.Equal(t, *semver.New("2.1.0-rc.1"), *vers) vers, err = ExtractTiDBVersion("5.7.10-TiDB-v2.0.4-1-g06a0bf5") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("2.0.4")) + require.NoError(t, err) + require.Equal(t, *semver.New("2.0.4"), *vers) vers, err = ExtractTiDBVersion("5.7.10-TiDB-v2.0.7") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("2.0.7")) + require.NoError(t, err) + require.Equal(t, *semver.New("2.0.7"), *vers) vers, err = ExtractTiDBVersion("8.0.12-TiDB-v3.0.5-beta.12") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("3.0.5-beta.12")) + require.NoError(t, err) + require.Equal(t, *semver.New("3.0.5-beta.12"), *vers) vers, err = ExtractTiDBVersion("5.7.25-TiDB-v3.0.0-beta-211-g09beefbe0-dirty") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("3.0.0-beta")) + require.NoError(t, err) + require.Equal(t, *semver.New("3.0.0-beta"), *vers) vers, err = ExtractTiDBVersion("8.0.12-TiDB-v3.0.5-dirty") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("3.0.5")) + require.NoError(t, err) + require.Equal(t, *semver.New("3.0.5"), *vers) vers, err = ExtractTiDBVersion("8.0.12-TiDB-v3.0.5-beta.12-dirty") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("3.0.5-beta.12")) + require.NoError(t, err) + require.Equal(t, *semver.New("3.0.5-beta.12"), *vers) vers, err = ExtractTiDBVersion("5.7.10-TiDB-v2.1.0-rc.1-7-g38c939f-dirty") - c.Assert(err, IsNil) - c.Assert(*vers, Equals, *semver.New("2.1.0-rc.1")) + require.NoError(t, err) + require.Equal(t, *semver.New("2.1.0-rc.1"), *vers) _, err = ExtractTiDBVersion("") - c.Assert(err, ErrorMatches, "not a valid TiDB version.*") + require.Error(t, err) + require.Regexp(t, "not a valid TiDB version.*", err.Error()) _, err = ExtractTiDBVersion("8.0.12") - c.Assert(err, ErrorMatches, "not a valid TiDB version.*") + require.Error(t, err) + require.Regexp(t, "not a valid TiDB version.*", err.Error()) _, err = ExtractTiDBVersion("not-a-valid-version") - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *checkSuite) TestCheckVersion(c *C) { +func TestCheckVersion(t *testing.T) { + t.Parallel() + err := CheckVersion("TiNB", *semver.New("2.3.5"), *semver.New("2.1.0"), *semver.New("3.0.0")) - c.Assert(err, IsNil) + require.NoError(t, err) err = CheckVersion("TiNB", *semver.New("2.1.0"), *semver.New("2.3.5"), *semver.New("3.0.0")) - c.Assert(err, ErrorMatches, "TiNB version too old.*") + require.Error(t, err) + require.Regexp(t, "TiNB version too old.*", err.Error()) err = CheckVersion("TiNB", *semver.New("3.1.0"), *semver.New("2.3.5"), *semver.New("3.0.0")) - c.Assert(err, ErrorMatches, "TiNB version too new.*") + require.Error(t, err) + require.Regexp(t, "TiNB version too new.*", err.Error()) err = CheckVersion("TiNB", *semver.New("3.0.0-beta"), *semver.New("2.3.5"), *semver.New("3.0.0")) - c.Assert(err, ErrorMatches, "TiNB version too new.*") + require.Error(t, err) + require.Regexp(t, "TiNB version too new.*", err.Error()) } -type versionEqualsC struct{} - -func (v versionEqualsC) Info() *CheckerInfo { - return &CheckerInfo{ - Name: "VersionEquals", - Params: []string{"source", "target"}, - } -} - -func (v versionEqualsC) Check(params []interface{}, names []string) (result bool, error string) { - source := params[0].(*semver.Version) - target := params[1].(*semver.Version) +func versionEqualCheck(source *semver.Version, target *semver.Version) (result bool) { if source == nil || target == nil { - if target == source { - return true, "" - } - return false, fmt.Sprintf("one of version is nil but another is not (%s and %s)", params[0], params[1]) + return target == source } - if source.Equal(*target) { - return true, "" - } - return false, fmt.Sprintf("version not equal (%s vs %s)", source, target) + return source.Equal(*target) } -var versionEquals versionEqualsC +func TestNormalizeBackupVersion(t *testing.T) { + t.Parallel() -func (s *checkSuite) TestNormalizeBackupVersion(c *C) { cases := []struct { target string source string @@ -294,13 +291,15 @@ func (s *checkSuite) TestNormalizeBackupVersion(c *C) { for _, testCase := range cases { target, _ := semver.NewVersion(testCase.target) source := NormalizeBackupVersion(testCase.source) - c.Assert(source, versionEquals, target) + result := versionEqualCheck(source, target) + require.Truef(t, result, "source=%v, target=%v", source, target) } } -func (s *checkSuite) TestDetectServerInfo(c *C) { +func TestDetectServerInfo(t *testing.T) { + t.Parallel() db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) defer db.Close() mkVer := makeVersion @@ -320,7 +319,7 @@ func (s *checkSuite) TestDetectServerInfo(c *C) { for _, datum := range data { tag, r, serverTp, expectVer := dec(datum) - cmt := Commentf("test case number: %d", tag) + cmt := fmt.Sprintf("test case number: %d", tag) tidbVersionQuery := mock.ExpectQuery("SELECT tidb_version\\(\\);") if strings.HasPrefix(r, "Release Version:") { @@ -332,17 +331,18 @@ func (s *checkSuite) TestDetectServerInfo(c *C) { } verStr, err := FetchVersion(context.Background(), db) - c.Assert(err, IsNil, cmt) + require.NoError(t, err, cmt) + info := ParseServerInfo(verStr) - c.Assert(info.ServerType, Equals, serverTp, cmt) - c.Assert(info.ServerVersion == nil, Equals, expectVer == nil, cmt) + require.Equal(t, serverTp, info.ServerType, cmt) + require.Equal(t, expectVer == nil, info.ServerVersion == nil, cmt) if info.ServerVersion == nil { - c.Assert(expectVer, IsNil, cmt) + require.Nil(t, expectVer, cmt) } else { fmt.Printf("%v, %v\n", *info.ServerVersion, *expectVer) - c.Assert(info.ServerVersion.Equal(*expectVer), IsTrue) + require.True(t, info.ServerVersion.Equal(*expectVer)) } - c.Assert(mock.ExpectationsWereMet(), IsNil, cmt) + require.NoError(t, mock.ExpectationsWereMet(), cmt) } } func makeVersion(major, minor, patch int64, preRelease string) *semver.Version { @@ -355,9 +355,9 @@ func makeVersion(major, minor, patch int64, preRelease string) *semver.Version { } } -func (s *checkSuite) TestFetchVersion(c *C) { +func TestFetchVersion(t *testing.T) { db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) tidbVersion := `Release Version: v5.2.1 Edition: Community @@ -373,20 +373,21 @@ Check Table Before Drop: false` mock.ExpectQuery("SELECT tidb_version\\(\\);").WillReturnRows(sqlmock. NewRows([]string{""}).AddRow(tidbVersion)) versionStr, err := FetchVersion(ctx, db) - c.Assert(err, IsNil) - c.Assert(versionStr, Equals, tidbVersion) + require.NoError(t, err) + require.Equal(t, tidbVersion, versionStr) mock.ExpectQuery("SELECT tidb_version\\(\\);").WillReturnError(errors.New("mock failure")) mock.ExpectQuery("SELECT version\\(\\);").WillReturnRows(sqlmock. NewRows([]string{""}).AddRow("5.7.25")) versionStr, err = FetchVersion(ctx, db) - c.Assert(err, IsNil) - c.Assert(versionStr, Equals, "5.7.25") + require.NoError(t, err) + require.Equal(t, "5.7.25", versionStr) mock.ExpectQuery("SELECT tidb_version\\(\\);").WillReturnError(errors.New("mock failure")) mock.ExpectQuery("SELECT version\\(\\);").WillReturnError(errors.New("mock failure")) _, err = FetchVersion(ctx, db) - c.Assert(err, ErrorMatches, ".*mock failure") + require.Error(t, err) + require.Regexp(t, ".*mock failure", err.Error()) } diff --git a/br/tests/lightning_distributed_import/config.toml b/br/tests/lightning_distributed_import/config.toml new file mode 100644 index 0000000000000..200af8e45dfdc --- /dev/null +++ b/br/tests/lightning_distributed_import/config.toml @@ -0,0 +1,9 @@ +[tikv-importer] +backend = 'local' +duplicate-resolution = 'none' + +[post-restore] +checksum = "required" + +[mydumper.csv] +header = false diff --git a/br/tests/lightning_distributed_import/data1/distributed_import-schema-create.sql b/br/tests/lightning_distributed_import/data1/distributed_import-schema-create.sql new file mode 100644 index 0000000000000..19c586879a959 --- /dev/null +++ b/br/tests/lightning_distributed_import/data1/distributed_import-schema-create.sql @@ -0,0 +1 @@ +create database distributed_import; diff --git a/br/tests/lightning_distributed_import/data1/distributed_import.t-schema.sql b/br/tests/lightning_distributed_import/data1/distributed_import.t-schema.sql new file mode 100644 index 0000000000000..7cf7f72809d33 --- /dev/null +++ b/br/tests/lightning_distributed_import/data1/distributed_import.t-schema.sql @@ -0,0 +1 @@ +create table t(a int primary key, b varchar(255), c double); diff --git a/br/tests/lightning_distributed_import/data1/distributed_import.t.csv b/br/tests/lightning_distributed_import/data1/distributed_import.t.csv new file mode 100644 index 0000000000000..7ee53cee3916e --- /dev/null +++ b/br/tests/lightning_distributed_import/data1/distributed_import.t.csv @@ -0,0 +1,5 @@ +1,a1,1.1 +3,b3,3.3 +5,c5,5.5 +7,d7,7.7 +9,e9,9.9 diff --git a/br/tests/lightning_distributed_import/data2/distributed_import-schema-create.sql b/br/tests/lightning_distributed_import/data2/distributed_import-schema-create.sql new file mode 100644 index 0000000000000..19c586879a959 --- /dev/null +++ b/br/tests/lightning_distributed_import/data2/distributed_import-schema-create.sql @@ -0,0 +1 @@ +create database distributed_import; diff --git a/br/tests/lightning_distributed_import/data2/distributed_import.t-schema.sql b/br/tests/lightning_distributed_import/data2/distributed_import.t-schema.sql new file mode 100644 index 0000000000000..7cf7f72809d33 --- /dev/null +++ b/br/tests/lightning_distributed_import/data2/distributed_import.t-schema.sql @@ -0,0 +1 @@ +create table t(a int primary key, b varchar(255), c double); diff --git a/br/tests/lightning_distributed_import/data2/distributed_import.t.csv b/br/tests/lightning_distributed_import/data2/distributed_import.t.csv new file mode 100644 index 0000000000000..1baadab31feac --- /dev/null +++ b/br/tests/lightning_distributed_import/data2/distributed_import.t.csv @@ -0,0 +1,5 @@ +2,a2,2.2 +4,b4,4.4 +6,c6,6.6 +8,d8,8.8 +10,e10,10.10 diff --git a/br/tests/lightning_distributed_import/run.sh b/br/tests/lightning_distributed_import/run.sh new file mode 100644 index 0000000000000..f640ec3159c75 --- /dev/null +++ b/br/tests/lightning_distributed_import/run.sh @@ -0,0 +1,36 @@ +#!/bin/bash +# +# 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. + +set -eux + +LOG_FILE1="$TEST_DIR/lightning-distributed-import1.log" +LOG_FILE2="$TEST_DIR/lightning-distributed-import2.log" + +# let lightning run a bit slow to avoid some table in the first lightning finish too fast. +export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/lightning/restore/SlowDownImport=sleep(50)" + +run_lightning --backend local --sorted-kv-dir "$TEST_DIR/lightning_distributed_import.sorted1" \ + -d "tests/$TEST_NAME/data1" --log-file "$LOG_FILE1" --config "tests/$TEST_NAME/config.toml" & +pid1="$!" + +run_lightning --backend local --sorted-kv-dir "$TEST_DIR/lightning_distributed_import.sorted2" \ + -d "tests/$TEST_NAME/data2" --log-file "$LOG_FILE2" --config "tests/$TEST_NAME/config.toml" & +pid2="$!" + +wait "$pid1" "$pid2" + +run_sql 'select count(*) from distributed_import.t' +check_contains 'count(*): 10' diff --git a/cmd/explaintest/r/new_character_set_builtin.result b/cmd/explaintest/r/new_character_set_builtin.result index 9611c63e21b8d..b4944f3057611 100644 --- a/cmd/explaintest/r/new_character_set_builtin.result +++ b/cmd/explaintest/r/new_character_set_builtin.result @@ -215,3 +215,63 @@ select sha2(a, 514), sha2(b, 514), sha2(c, 514) from t; sha2(a, 514) sha2(b, 514) sha2(c, 514) NULL NULL NULL set @@tidb_enable_vectorized_expression = false; +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 ('一二三', '一二三', '一二三'); +select md5(b) from t where md5(b) = 'a45d4af7b243e7f393fa09bed72ac73e'; +md5(b) +a45d4af7b243e7f393fa09bed72ac73e +set @@tidb_enable_vectorized_expression = true; +select md5(b) from t where md5(b) = 'a45d4af7b243e7f393fa09bed72ac73e'; +md5(b) +a45d4af7b243e7f393fa09bed72ac73e +set @@tidb_enable_vectorized_expression = false; +drop table if exists t; +create table t (a char(20)); +insert into t values ('65'), ('123456'), ('123456789'); +select char(a using gbk), char(a using utf8), char(a) from t; +char(a using gbk) char(a using utf8) char(a) +A A A +釦 @ @ +NULL [ [ +select char(12345678 using gbk); +char(12345678 using gbk) +糰N +set @@tidb_enable_vectorized_expression = true; +select char(a using gbk), char(a using utf8), char(a) from t; +char(a using gbk) char(a using utf8) char(a) +A A A +釦 @ @ +NULL [ [ +select char(12345678 using gbk); +char(12345678 using gbk) +糰N +set @@tidb_enable_vectorized_expression = false; +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 ('一二三', '一二三', '一二三'); +select md5(compress(a)), md5(compress(b)), md5(compress(c)) from t; +md5(compress(a)) md5(compress(b)) md5(compress(c)) +2198d4d3b06a6cba3f9275c7e364105c 5e587a14393aecf0629bb29dbd6b4379 905068487b6220f70bb71a48323826be +set @@tidb_enable_vectorized_expression = true; +select md5(compress(a)), md5(compress(b)), md5(compress(c)) from t; +md5(compress(a)) md5(compress(b)) md5(compress(c)) +2198d4d3b06a6cba3f9275c7e364105c 5e587a14393aecf0629bb29dbd6b4379 905068487b6220f70bb71a48323826be +set @@tidb_enable_vectorized_expression = false; +set @@block_encryption_mode='aes-128-ecb'; +select hex(aes_decrypt(0xC54279F381B0710E145E94106F03C94C, '123')) as a, hex(aes_decrypt(0x7A747EC6F1906276D036B1F3CE27BAAB, '123')) as b; +a b +E4B880E4BA8CE4B889 D2BBB6FEC8FD +set @@block_encryption_mode='aes-128-ofb'; +select hex(aes_decrypt(0x91C44DE866D0745252, '1234567890123456', '1234567890123456')) as a, hex(aes_decrypt(0xA7C77BF214A1, '1234567890123456', '1234567890123456')) as b; +a b +E4B880E4BA8CE4B889 D2BBB6FEC8FD +set @@tidb_enable_vectorized_expression = true; +select hex(aes_decrypt(0x91C44DE866D0745252, '1234567890123456', '1234567890123456')) as a, hex(aes_decrypt(0xA7C77BF214A1, '1234567890123456', '1234567890123456')) as b; +a b +E4B880E4BA8CE4B889 D2BBB6FEC8FD +set @@block_encryption_mode='aes-128-ecb'; +select hex(aes_decrypt(0xC54279F381B0710E145E94106F03C94C, '123')) as a, hex(aes_decrypt(0x7A747EC6F1906276D036B1F3CE27BAAB, '123')) as b; +a b +E4B880E4BA8CE4B889 D2BBB6FEC8FD +set @@tidb_enable_vectorized_expression = false; diff --git a/cmd/explaintest/t/new_character_set_builtin.test b/cmd/explaintest/t/new_character_set_builtin.test index 0f2b7332fe75a..0c6fefac7919d 100644 --- a/cmd/explaintest/t/new_character_set_builtin.test +++ b/cmd/explaintest/t/new_character_set_builtin.test @@ -112,3 +112,43 @@ select sha2(a, 256), sha2(b, 256), sha2(c, 256) from t; select sha2(a, 384), sha2(b, 384), sha2(c, 384) from t; select sha2(a, 514), sha2(b, 514), sha2(c, 514) from t; set @@tidb_enable_vectorized_expression = false; + +-- test for push md5() builtin function down to unistore +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 ('一二三', '一二三', '一二三'); +select md5(b) from t where md5(b) = 'a45d4af7b243e7f393fa09bed72ac73e'; +set @@tidb_enable_vectorized_expression = true; +select md5(b) from t where md5(b) = 'a45d4af7b243e7f393fa09bed72ac73e'; +set @@tidb_enable_vectorized_expression = false; + +-- test for builtin function char() +drop table if exists t; +create table t (a char(20)); +insert into t values ('65'), ('123456'), ('123456789'); +select char(a using gbk), char(a using utf8), char(a) from t; +select char(12345678 using gbk); +set @@tidb_enable_vectorized_expression = true; +select char(a using gbk), char(a using utf8), char(a) from t; +select char(12345678 using gbk); +set @@tidb_enable_vectorized_expression = false; + +-- test for builtin function compress() +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 ('一二三', '一二三', '一二三'); +select md5(compress(a)), md5(compress(b)), md5(compress(c)) from t; +set @@tidb_enable_vectorized_expression = true; +select md5(compress(a)), md5(compress(b)), md5(compress(c)) from t; +set @@tidb_enable_vectorized_expression = false; + +-- test for builtin function aes_decrypt() +set @@block_encryption_mode='aes-128-ecb'; +select hex(aes_decrypt(0xC54279F381B0710E145E94106F03C94C, '123')) as a, hex(aes_decrypt(0x7A747EC6F1906276D036B1F3CE27BAAB, '123')) as b; +set @@block_encryption_mode='aes-128-ofb'; +select hex(aes_decrypt(0x91C44DE866D0745252, '1234567890123456', '1234567890123456')) as a, hex(aes_decrypt(0xA7C77BF214A1, '1234567890123456', '1234567890123456')) as b; +set @@tidb_enable_vectorized_expression = true; +select hex(aes_decrypt(0x91C44DE866D0745252, '1234567890123456', '1234567890123456')) as a, hex(aes_decrypt(0xA7C77BF214A1, '1234567890123456', '1234567890123456')) as b; +set @@block_encryption_mode='aes-128-ecb'; +select hex(aes_decrypt(0xC54279F381B0710E145E94106F03C94C, '123')) as a, hex(aes_decrypt(0x7A747EC6F1906276D036B1F3CE27BAAB, '123')) as b; +set @@tidb_enable_vectorized_expression = false; diff --git a/ddl/attributes_sql_test.go b/ddl/attributes_sql_test.go index 58511b4cb9fa3..c3295b1d518d0 100644 --- a/ddl/attributes_sql_test.go +++ b/ddl/attributes_sql_test.go @@ -15,20 +15,31 @@ package ddl_test import ( + "context" "fmt" + "math" . "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/util/gcutil" "github.com/pingcap/tidb/util/testkit" ) -func (s *testDBSuite8) TestAlterTableAttributes(c *C) { +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() @@ -36,32 +47,34 @@ func (s *testDBSuite8) TestAlterTableAttributes(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int);`) + tk.MustExec(`create table alter_t (c int);`) // normal cases - _, err = tk.Exec(`alter table t1 attributes="merge_option=allow";`) + _, err = tk.Exec(`alter table alter_t attributes="merge_option=allow";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 attributes="merge_option=allow,key=value";`) + _, err = tk.Exec(`alter table alter_t attributes="merge_option=allow,key=value";`) c.Assert(err, IsNil) // space cases - _, err = tk.Exec(`alter table t1 attributes=" merge_option=allow ";`) + _, err = tk.Exec(`alter table alter_t attributes=" merge_option=allow ";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 attributes=" merge_option = allow , key = value ";`) + _, err = tk.Exec(`alter table alter_t attributes=" merge_option = allow , key = value ";`) c.Assert(err, IsNil) // without equal - _, err = tk.Exec(`alter table t1 attributes " merge_option=allow ";`) + _, err = tk.Exec(`alter table alter_t attributes " merge_option=allow ";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 attributes " merge_option=allow , key=value ";`) + _, err = tk.Exec(`alter table alter_t attributes " merge_option=allow , key=value ";`) c.Assert(err, IsNil) } -func (s *testDBSuite8) TestAlterTablePartitionAttributes(c *C) { +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() @@ -69,7 +82,7 @@ func (s *testDBSuite8) TestAlterTablePartitionAttributes(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table alter_p (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), @@ -78,29 +91,31 @@ PARTITION BY RANGE (c) ( );`) // normal cases - _, err = tk.Exec(`alter table t1 partition p0 attributes="merge_option=allow";`) + _, err = tk.Exec(`alter table alter_p partition p0 attributes="merge_option=allow";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p1 attributes="merge_option=allow,key=value";`) + _, err = tk.Exec(`alter table alter_p partition p1 attributes="merge_option=allow,key=value";`) c.Assert(err, IsNil) // space cases - _, err = tk.Exec(`alter table t1 partition p2 attributes=" merge_option=allow ";`) + _, err = tk.Exec(`alter table alter_p partition p2 attributes=" merge_option=allow ";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p3 attributes=" merge_option = allow , key = value ";`) + _, err = tk.Exec(`alter table alter_p partition p3 attributes=" merge_option = allow , key = value ";`) c.Assert(err, IsNil) // without equal - _, err = tk.Exec(`alter table t1 partition p1 attributes " merge_option=allow ";`) + _, err = tk.Exec(`alter table alter_p partition p1 attributes " merge_option=allow ";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p1 attributes " merge_option=allow , key=value ";`) + _, err = tk.Exec(`alter table alter_p partition p1 attributes " merge_option=allow , key=value ";`) c.Assert(err, IsNil) } -func (s *testDBSuite8) TestTruncateTable(c *C) { +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() @@ -108,57 +123,59 @@ func (s *testDBSuite8) TestTruncateTable(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table truncate_t (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11) );`) - // add rules - _, err = tk.Exec(`alter table t1 attributes="key=value";`) + // add attributes + _, err = tk.Exec(`alter table truncate_t attributes="key=value";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p0 attributes="key1=value1";`) + _, err = tk.Exec(`alter table truncate_t partition p0 attributes="key1=value1";`) c.Assert(err, IsNil) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows), Equals, 2) // truncate table - _, err = tk.Exec(`truncate table t1;`) + _, err = tk.Exec(`truncate table truncate_t;`) c.Assert(err, IsNil) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows1), Equals, 2) - // check table t1's attribute - c.Assert(rows1[0][0], Equals, "schema/test/t1") + // 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]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/t1/p0") + 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]) // test only table - tk.MustExec(`create table t2 (c int);`) + tk.MustExec(`create table truncate_ot (c int);`) // add attribute - _, err = tk.Exec(`alter table t2 attributes="key=value";`) + _, err = tk.Exec(`alter table truncate_ot attributes="key=value";`) c.Assert(err, IsNil) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows2), Equals, 3) // truncate table - _, err = tk.Exec(`truncate table t2;`) + _, err = tk.Exec(`truncate table truncate_ot;`) c.Assert(err, IsNil) rows3 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows3), Equals, 3) - // check table t1's attribute - c.Assert(rows3[2][0], Equals, "schema/test/t2") - c.Assert(rows3[2][2], Equals, `"key=value"`) - c.Assert(rows3[2][3], Not(Equals), rows2[2][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]) } -func (s *testDBSuite8) TestRenameTable(c *C) { +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() @@ -166,57 +183,59 @@ func (s *testDBSuite8) TestRenameTable(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table rename_t (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11) );`) - // add rules - _, err = tk.Exec(`alter table t1 attributes="key=value";`) + // add attributes + _, err = tk.Exec(`alter table rename_t attributes="key=value";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p0 attributes="key1=value1";`) + _, err = tk.Exec(`alter table rename_t partition p0 attributes="key1=value1";`) c.Assert(err, IsNil) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows), Equals, 2) // rename table - _, err = tk.Exec(`rename table t1 to t2;`) + _, err = tk.Exec(`rename table rename_t to rename_t1;`) c.Assert(err, IsNil) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows1), Equals, 2) - // check table t2's attribute - c.Assert(rows1[0][0], Equals, "schema/test/t2") + // 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]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/t2/p0") + 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]) // test only table - tk.MustExec(`create table t3 (c int);`) + tk.MustExec(`create table rename_ot (c int);`) // add attribute - _, err = tk.Exec(`alter table t3 attributes="key=value";`) + _, err = tk.Exec(`alter table rename_ot attributes="key=value";`) c.Assert(err, IsNil) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows2), Equals, 3) // rename table - _, err = tk.Exec(`rename table t3 to t4;`) + _, err = tk.Exec(`rename table rename_ot to rename_ot1;`) c.Assert(err, IsNil) rows3 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows3), Equals, 3) - // check table t4's attribute - c.Assert(rows3[2][0], Equals, "schema/test/t4") - c.Assert(rows3[2][2], Equals, `"key=value"`) - c.Assert(rows3[2][3], Equals, rows2[2][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]) } -func (s *testDBSuite8) TestRecoverTable(c *C) { +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() @@ -224,7 +243,7 @@ func (s *testDBSuite8) TestRecoverTable(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table recover_t (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11) @@ -239,36 +258,38 @@ PARTITION BY RANGE (c) ( err = gcutil.EnableGC(tk.Se) c.Assert(err, IsNil) - // add rules - _, err = tk.Exec(`alter table t1 attributes="key=value";`) + // add attributes + _, err = tk.Exec(`alter table recover_t attributes="key=value";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p0 attributes="key1=value1";`) + _, err = tk.Exec(`alter table recover_t partition p0 attributes="key1=value1";`) c.Assert(err, IsNil) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows), Equals, 2) // drop table - _, err = tk.Exec(`drop table t1;`) + _, err = tk.Exec(`drop table recover_t;`) c.Assert(err, IsNil) // recover table - _, err = tk.Exec(`recover table t1;`) + _, err = tk.Exec(`recover table recover_t;`) c.Assert(err, IsNil) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows1), Equals, 2) - // check table t1's attribute - c.Assert(rows1[0][0], Equals, "schema/test/t1") + // 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]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/t1/p0") + 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]) } -func (s *testDBSuite8) TestFlashbackTable(c *C) { +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() @@ -276,7 +297,7 @@ func (s *testDBSuite8) TestFlashbackTable(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table flash_t (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11) @@ -291,53 +312,108 @@ PARTITION BY RANGE (c) ( err = gcutil.EnableGC(tk.Se) c.Assert(err, IsNil) - // add rules - _, err = tk.Exec(`alter table t1 attributes="key=value";`) + // add attributes + _, err = tk.Exec(`alter table flash_t attributes="key=value";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p0 attributes="key1=value1";`) + _, err = tk.Exec(`alter table flash_t partition p0 attributes="key1=value1";`) c.Assert(err, IsNil) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows), Equals, 2) // drop table - _, err = tk.Exec(`drop table t1;`) + _, err = tk.Exec(`drop table flash_t;`) c.Assert(err, IsNil) // flashback table - _, err = tk.Exec(`flashback table t1 to t2;`) + _, err = tk.Exec(`flashback table flash_t to flash_t1;`) c.Assert(err, IsNil) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows1), Equals, 2) - // check table t2's attribute - c.Assert(rows1[0][0], Equals, "schema/test/t2") + // 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]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/t2/p0") + 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]) // truncate table - _, err = tk.Exec(`truncate table t2;`) + _, err = tk.Exec(`truncate table flash_t1;`) c.Assert(err, IsNil) // flashback table - _, err = tk.Exec(`flashback table t2 to t3;`) + _, err = tk.Exec(`flashback table flash_t1 to flash_t2;`) c.Assert(err, IsNil) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows1), Equals, 2) - // check table t3's attribute - c.Assert(rows2[0][0], Equals, "schema/test/t3") + // 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]) // check partition p0's attribute - c.Assert(rows2[1][0], Equals, "schema/test/t3/p0") + 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]) } -func (s *testDBSuite8) TestPartition(c *C) { +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) + tk.MustExec("use test") + tk.MustExec(`create table drop_t (c int) +PARTITION BY RANGE (c) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11) +);`) + failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + defer func() { + failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") + }() + + timeBeforeDrop, _, safePointSQL, resetGC := testkit.MockGC(tk) + defer resetGC() + + // Set GC safe point + tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) + // Set GC enable. + err = gcutil.EnableGC(tk.Se) + c.Assert(err, IsNil) + + gcWorker, err := gcworker.NewMockGCWorker(store) + c.Assert(err, IsNil) + + // 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) + rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 2) + // drop table + _, err = tk.Exec(`drop table drop_t;`) + c.Assert(err, IsNil) + + err = gcWorker.DeleteRanges(context.Background(), uint64(math.MaxInt64)) + c.Assert(err, IsNil) + rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 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() @@ -345,70 +421,149 @@ func (s *testDBSuite8) TestPartition(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table recreate_t (c int) +PARTITION BY RANGE (c) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11) +);`) + failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + defer func() { + failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") + }() + + timeBeforeDrop, _, safePointSQL, resetGC := testkit.MockGC(tk) + defer resetGC() + + // Set GC safe point + tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) + // Set GC enable. + err = gcutil.EnableGC(tk.Se) + c.Assert(err, IsNil) + + gcWorker, err := gcworker.NewMockGCWorker(store) + c.Assert(err, IsNil) + + // 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) + rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 2) + // drop table + _, err = tk.Exec(`drop table recreate_t;`) + c.Assert(err, IsNil) + + rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 2) + + tk.MustExec(`create table recreate_t (c int) + PARTITION BY RANGE (c) ( + PARTITION p0 VALUES LESS THAN (6), + 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) + rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 3) + + err = gcWorker.DeleteRanges(context.Background(), uint64(math.MaxInt64)) + c.Assert(err, IsNil) + rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 2) + + // drop table + _, err = tk.Exec(`drop table recreate_t;`) + c.Assert(err, IsNil) + err = gcWorker.DeleteRanges(context.Background(), uint64(math.MaxInt64)) + c.Assert(err, IsNil) + rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 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) + tk.MustExec("use test") + tk.MustExec(`create table part (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (20) );`) - tk.MustExec(`create table t2 (c int);`) + tk.MustExec(`create table part1 (c int);`) - // add rules - _, err = tk.Exec(`alter table t1 attributes="key=value";`) + // add attributes + _, err = tk.Exec(`alter table part attributes="key=value";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p0 attributes="key1=value1";`) + _, err = tk.Exec(`alter table part partition p0 attributes="key1=value1";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p1 attributes="key2=value2";`) + _, err = tk.Exec(`alter table part partition p1 attributes="key2=value2";`) c.Assert(err, IsNil) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows), Equals, 3) // drop partition // partition p0's attribute will be deleted - _, err = tk.Exec(`alter table t1 drop partition p0;`) + _, err = tk.Exec(`alter table part drop partition p0;`) c.Assert(err, IsNil) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows1), Equals, 2) - c.Assert(rows1[0][0], Equals, "schema/test/t1") + 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/t1/p1") + 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]) // truncate partition // partition p1's key range will be updated - _, err = tk.Exec(`alter table t1 truncate partition p1;`) + _, err = tk.Exec(`alter table part truncate partition p1;`) c.Assert(err, IsNil) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows2), Equals, 2) - c.Assert(rows2[0][0], Equals, "schema/test/t1") + 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/t1/p1") + 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]) // exchange partition - // partition p1's attribute will be exchanged to table t2 + // 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 t1 exchange partition p1 with table t2;`) + _, err = tk.Exec(`alter table part exchange partition p1 with table part1;`) c.Assert(err, IsNil) rows3 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() c.Assert(len(rows3), Equals, 2) - c.Assert(rows3[0][0], Equals, "schema/test/t1") + 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/t2") + 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]) } -func (s *testDBSuite8) TestDropSchema(c *C) { +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() @@ -416,19 +571,19 @@ func (s *testDBSuite8) TestDropSchema(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table drop_s1 (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11) );`) - tk.MustExec(`create table t2 (c int);`) + tk.MustExec(`create table drop_s2 (c int);`) - // add rules - _, err = tk.Exec(`alter table t1 attributes="key=value";`) + // add attributes + _, err = tk.Exec(`alter table drop_s1 attributes="key=value";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p0 attributes="key1=value1";`) + _, err = tk.Exec(`alter table drop_s1 partition p0 attributes="key1=value1";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t2 attributes="key=value";`) + _, err = tk.Exec(`alter table drop_s2 attributes="key=value";`) c.Assert(err, IsNil) rows := tk.MustQuery(`select * from information_schema.attributes;`).Rows() c.Assert(len(rows), Equals, 3) @@ -439,11 +594,13 @@ PARTITION BY RANGE (c) ( c.Assert(len(rows), Equals, 0) } -func (s *testDBSuite8) TestDefaultKeyword(c *C) { +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() @@ -451,26 +608,26 @@ func (s *testDBSuite8) TestDefaultKeyword(c *C) { }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") - tk.MustExec(`create table t1 (c int) + tk.MustExec(`create table def (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11) );`) - // add rules - _, err = tk.Exec(`alter table t1 attributes="key=value";`) + // add attributes + _, err = tk.Exec(`alter table def attributes="key=value";`) c.Assert(err, IsNil) - _, err = tk.Exec(`alter table t1 partition p0 attributes="key1=value1";`) + _, err = tk.Exec(`alter table def partition p0 attributes="key1=value1";`) c.Assert(err, IsNil) rows := tk.MustQuery(`select * from information_schema.attributes;`).Rows() c.Assert(len(rows), Equals, 2) // reset the partition p0's attribute - _, err = tk.Exec(`alter table t1 partition p0 attributes=default;`) + _, err = tk.Exec(`alter table def partition p0 attributes=default;`) c.Assert(err, IsNil) rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows() c.Assert(len(rows), Equals, 1) - // reset the table t1's attribute - _, err = tk.Exec(`alter table t1 attributes=default;`) + // reset the table def's attribute + _, err = tk.Exec(`alter table def attributes=default;`) c.Assert(err, IsNil) rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows() c.Assert(len(rows), Equals, 0) diff --git a/ddl/db_cache_test.go b/ddl/db_cache_test.go index bc51630e926c7..e92045ae131f2 100644 --- a/ddl/db_cache_test.go +++ b/ddl/db_cache_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util/testkit" ) @@ -35,7 +36,7 @@ func (s *testDBSuite2) TestAlterTableCache(c *C) { tk.MustGetErrCode("alter table t1 ca", errno.ErrParse) tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable) tk.MustExec("alter table t1 cache") - checkTableCache(c, tk.Se, "test", "t1") + checkTableCacheStatus(c, tk.Se, "test", "t1", model.TableCacheStatusEnable) tk.MustExec("drop table if exists t1") /*Test can't skip schema checker*/ tk.MustExec("drop table if exists t1,t2") @@ -110,3 +111,44 @@ func (s *testDBSuite2) TestAlterViewTableCache(c *C) { tk.MustExec("create view v as select * from cache_view_t") tk.MustGetErrCode("alter table v cache", errno.ErrWrongObject) } + +func (s *testDBSuite2) TestAlterTableNoCache(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists nocache_t1") + /* Test of cache table */ + tk.MustExec("create table nocache_t1 ( n int auto_increment primary key)") + tk.MustExec("alter table nocache_t1 cache") + checkTableCacheStatus(c, tk.Se, "test", "nocache_t1", model.TableCacheStatusEnable) + tk.MustExec("alter table nocache_t1 nocache") + checkTableCacheStatus(c, tk.Se, "test", "nocache_t1", model.TableCacheStatusDisable) + tk.MustExec("drop table if exists t1") + // Test if a table is not exists + tk.MustExec("drop table if exists nocache_t") + tk.MustGetErrCode("alter table nocache_t cache", errno.ErrNoSuchTable) + tk.MustExec("create table nocache_t (a int)") + tk.MustExec("alter table nocache_t nocache") + // Multiple no alter cache is okay + tk.MustExec("alter table nocache_t nocache") + tk.MustExec("alter table nocache_t nocache") +} + +func (s *testDBSuite2) TestIndexOnCacheTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + /*Test cache table can't add/drop/rename index */ + tk.MustExec("drop table if exists cache_index") + tk.MustExec("create table cache_index (c1 int primary key, c2 int, c3 int, index ok2(c2))") + defer tk.MustExec("drop table if exists cache_index") + tk.MustExec("alter table cache_index cache") + tk.MustGetErrCode("create index cache_c2 on cache_index(c2)", errno.ErrOptOnCacheTable) + tk.MustGetErrCode("alter table cache_index add index k2(c2)", errno.ErrOptOnCacheTable) + tk.MustGetErrCode("alter table cache_index drop index ok2", errno.ErrOptOnCacheTable) + /*Test rename index*/ + tk.MustGetErrCode("alter table cache_index rename index ok2 to ok", errno.ErrOptOnCacheTable) + /*Test drop different indexes*/ + tk.MustExec("drop table if exists cache_index_1") + tk.MustExec("create table cache_index_1 (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") + tk.MustExec("alter table cache_index_1 cache") + tk.MustGetErrCode("alter table cache_index_1 drop index i1, drop index i2;", errno.ErrOptOnCacheTable) +} diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 04c6f4f8a159e..da187483e8ea6 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -2644,26 +2644,17 @@ func testPartitionDropIndex(c *C, store kv.Storage, lease time.Duration, idxName tk.MustExec(addIdxSQL) ctx := tk.Se.(sessionctx.Context) - is := domain.GetDomain(ctx).InfoSchema() - t, err := is.TableByName(model.NewCIStr("test_db"), model.NewCIStr("partition_drop_idx")) - c.Assert(err, IsNil) - - var idx1 table.Index - for _, pidx := range t.Indices() { - if pidx.Meta().Name.L == idxName { - idx1 = pidx - break - } - } - c.Assert(idx1, NotNil) + indexID := testGetIndexID(c, ctx, "test_db", "partition_drop_idx", idxName) + jobIDExt, reset := setupJobIDExtCallback(ctx) + defer reset() testutil.SessionExecInGoroutine(store, dropIdxSQL, done) ticker := time.NewTicker(lease / 2) defer ticker.Stop() LOOP: for { select { - case err = <-done: + case err := <-done: if err == nil { break LOOP } @@ -2679,23 +2670,7 @@ LOOP: num += step } } - - is = domain.GetDomain(ctx).InfoSchema() - t, err = is.TableByName(model.NewCIStr("test_db"), model.NewCIStr("partition_drop_idx")) - c.Assert(err, IsNil) - // Only one partition id test is taken here. - pid := t.Meta().Partition.Definitions[0].ID - var idxn table.Index - t.Indices() - for _, idx := range t.Indices() { - if idx.Meta().Name.L == idxName { - idxn = idx - break - } - } - c.Assert(idxn, IsNil) - idx := tables.NewIndex(pid, t.Meta(), idx1.Meta()) - checkDelRangeDone(c, ctx, idx) + checkDelRangeAdded(tk, jobIDExt.jobID, indexID) tk.MustExec("drop table partition_drop_idx;") } @@ -2706,7 +2681,7 @@ func (s *testIntegrationSuite2) TestPartitionCancelAddPrimaryKey(c *C) { } func (s *testIntegrationSuite4) TestPartitionCancelAddIndex(c *C) { - idxName := "idx1" + idxName := "c3_index" addIdxSQL := "create unique index c3_index on t1 (c1)" testPartitionCancelAddIndex(c, s.store, s.dom.DDL(), s.lease, idxName, addIdxSQL) } @@ -2743,7 +2718,8 @@ func testPartitionCancelAddIndex(c *C, store kv.Storage, d ddl.DDL, lease time.D hook.OnJobUpdatedExported, c3IdxInfo, checkErr = backgroundExecOnJobUpdatedExported(c, store, ctx, hook, idxName) originHook := d.GetHook() defer d.(ddl.DDLForTest).SetHook(originHook) - d.(ddl.DDLForTest).SetHook(hook) + jobIDExt := wrapJobIDExtCallback(hook) + d.(ddl.DDLForTest).SetHook(jobIDExt) done := make(chan error, 1) go backgroundExec(store, addIdxSQL, done) @@ -2774,17 +2750,7 @@ LOOP: times++ } } - - t := testGetTableByName(c, ctx, "test_db", "t1") - // Only one partition id test is taken here. - pid := t.Meta().Partition.Definitions[0].ID - for _, tidx := range t.Indices() { - c.Assert(strings.EqualFold(tidx.Meta().Name.L, "c3_index"), IsFalse) - } - - idx := tables.NewIndex(pid, t.Meta(), c3IdxInfo) - checkDelRangeDone(c, ctx, idx) - + checkDelRangeAdded(tk, jobIDExt.jobID, c3IdxInfo.ID) tk.MustExec("drop table t1") } @@ -2800,7 +2766,7 @@ func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.C // When the job satisfies this case of addIndexNotFirstReorg, the worker will start to backfill indexes. if !addIndexNotFirstReorg { // Get the index's meta. - if c3IdxInfo != nil { + if c3IdxInfo.ID != 0 { return } t := testGetTableByName(c, ctx, "test_db", "t1") @@ -2809,7 +2775,7 @@ func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.C continue } if index.Meta().Name.L == idxName { - c3IdxInfo = index.Meta() + *c3IdxInfo = *index.Meta() } } return diff --git a/ddl/db_test.go b/ddl/db_test.go index d1eb22a08679c..cb1c90b54df49 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -573,7 +573,7 @@ func (s *testDBSuite8) TestCancelAddPrimaryKey(c *C) { } func (s *testDBSuite7) TestCancelAddIndex(c *C) { - idxName := "c3_index " + idxName := "c3_index" addIdxSQL := "create unique index c3_index on t1 (c3)" testCancelAddIndex(c, s.store, s.dom.DDL(), s.lease, idxName, addIdxSQL, "", s.dom) @@ -616,7 +616,8 @@ func testCancelAddIndex(c *C, store kv.Storage, d ddl.DDL, lease time.Duration, ctx := tk.Se.(sessionctx.Context) hook.OnJobUpdatedExported, c3IdxInfo, checkErr = backgroundExecOnJobUpdatedExported(c, store, ctx, hook, idxName) originalHook := d.GetHook() - d.(ddl.DDLForTest).SetHook(hook) + jobIDExt := wrapJobIDExtCallback(hook) + d.(ddl.DDLForTest).SetHook(jobIDExt) done := make(chan error, 1) go backgroundExec(store, addIdxSQL, done) @@ -646,14 +647,7 @@ LOOP: times++ } } - - t := testGetTableByName(c, ctx, "test_db", "t1") - for _, tidx := range t.Indices() { - c.Assert(strings.EqualFold(tidx.Meta().Name.L, idxName), IsFalse) - } - - idx := tables.NewIndex(t.Meta().ID, t.Meta(), c3IdxInfo) - checkDelRangeDone(c, ctx, idx) + checkDelRangeAdded(tk, jobIDExt.jobID, c3IdxInfo.ID) d.(ddl.DDLForTest).SetHook(originalHook) } @@ -1713,16 +1707,9 @@ func testDropIndex(c *C, store kv.Storage, lease time.Duration, createSQL, dropI tk.MustExec("insert into test_drop_index values (?, ?, ?)", i, i, i) } ctx := tk.Se.(sessionctx.Context) - t := testGetTableByName(c, ctx, "test_db", "test_drop_index") - var c3idx table.Index - for _, tidx := range t.Indices() { - if tidx.Meta().Name.L == idxName { - c3idx = tidx - break - } - } - c.Assert(c3idx, NotNil) - + indexID := testGetIndexID(c, ctx, "test_db", "test_drop_index", idxName) + jobIDExt, reset := setupJobIDExtCallback(ctx) + defer reset() testddlutil.SessionExecInGoroutine(store, dropIdxSQL, done) ticker := time.NewTicker(lease / 2) @@ -1750,20 +1737,7 @@ LOOP: rows := tk.MustQuery("explain select c1 from test_drop_index where c3 >= 0") c.Assert(strings.Contains(fmt.Sprintf("%v", rows), idxName), IsFalse) - // Check in index, it must be no index in KV. - // Make sure there is no index with name c3_index. - t = testGetTableByName(c, ctx, "test_db", "test_drop_index") - var nidx table.Index - for _, tidx := range t.Indices() { - if tidx.Meta().Name.L == idxName { - nidx = tidx - break - } - } - c.Assert(nidx, IsNil) - - idx := tables.NewIndex(t.Meta().ID, t.Meta(), c3idx.Meta()) - checkDelRangeDone(c, ctx, idx) + checkDelRangeAdded(tk, jobIDExt.jobID, indexID) tk.MustExec("drop table test_drop_index") } @@ -1821,19 +1795,14 @@ func (s *testDBSuite3) TestCancelDropColumn(c *C) { originalHook := s.dom.DDL().GetHook() s.dom.DDL().(ddl.DDLForTest).SetHook(hook) var err1 error - var c3idx table.Index for i := range testCases { + var c3IdxID int64 testCase = &testCases[i] if testCase.needAddColumn { s.mustExec(tk, c, "alter table test_drop_column add column c3 int") s.mustExec(tk, c, "alter table test_drop_column add index idx_c3(c3)") - tt := s.testGetTable(c, "test_drop_column") - for _, idx := range tt.Indices() { - if strings.EqualFold(idx.Meta().Name.L, "idx_c3") { - c3idx = idx - break - } - } + ctx := tk.Se.(sessionctx.Context) + c3IdxID = testGetIndexID(c, ctx, s.schemaName, "test_drop_column", "idx_c3") } _, err1 = tk.Exec("alter table test_drop_column drop column c3") var col1 *table.Column @@ -1864,9 +1833,10 @@ func (s *testDBSuite3) TestCancelDropColumn(c *C) { c.Assert(err1, IsNil) c.Assert(checkErr, NotNil) c.Assert(checkErr.Error(), Equals, admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID).Error()) - // Check index is deleted - ctx := s.s.(sessionctx.Context) - checkDelRangeDone(c, ctx, c3idx) + if c3IdxID != 0 { + // Check index is deleted + checkDelRangeAdded(tk, jobID, c3IdxID) + } } } s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) @@ -1928,19 +1898,14 @@ func (s *testDBSuite3) TestCancelDropColumns(c *C) { originalHook := s.dom.DDL().GetHook() s.dom.DDL().(ddl.DDLForTest).SetHook(hook) var err1 error - var c3idx table.Index for i := range testCases { + var c3IdxID int64 testCase = &testCases[i] if testCase.needAddColumn { s.mustExec(tk, c, "alter table test_drop_column add column c3 int, add column c4 int") s.mustExec(tk, c, "alter table test_drop_column add index idx_c3(c3)") - tt := s.testGetTable(c, "test_drop_column") - for _, idx := range tt.Indices() { - if strings.EqualFold(idx.Meta().Name.L, "idx_c3") { - c3idx = idx - break - } - } + ctx := tk.Se.(sessionctx.Context) + c3IdxID = testGetIndexID(c, ctx, s.schemaName, "test_drop_column", "idx_c3") } _, err1 = tk.Exec("alter table test_drop_column drop column c3, drop column c4") t := s.testGetTable(c, "test_drop_column") @@ -1969,9 +1934,10 @@ func (s *testDBSuite3) TestCancelDropColumns(c *C) { c.Assert(err1, IsNil) c.Assert(checkErr, NotNil) c.Assert(checkErr.Error(), Equals, admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID).Error()) - // Check index is deleted - ctx := s.s.(sessionctx.Context) - checkDelRangeDone(c, ctx, c3idx) + if c3IdxID != 0 { + // Check index is deleted + checkDelRangeAdded(tk, jobID, c3IdxID) + } } } s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) @@ -1979,47 +1945,56 @@ func (s *testDBSuite3) TestCancelDropColumns(c *C) { s.mustExec(tk, c, "alter table test_drop_column drop column c3, drop column c4") } -func checkDelRangeDone(c *C, ctx sessionctx.Context, idx table.Index) { - startTime := time.Now() - f := func() map[int64]struct{} { - handles := make(map[int64]struct{}) +func testGetIndexID(c *C, ctx sessionctx.Context, dbName, tblName, idxName string) int64 { + is := domain.GetDomain(ctx).InfoSchema() + t, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) + c.Assert(err, IsNil) - 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) - }() + for _, idx := range t.Indices() { + if idx.Meta().Name.L == idxName { + return idx.Meta().ID + } + } + c.Fatalf("index %s not found(db: %s, tbl: %s)", idxName, dbName, tblName) + return -1 +} - txn, err = ctx.Txn(true) - c.Assert(err, IsNil) - it, err := idx.SeekFirst(txn) - c.Assert(err, IsNil) - defer it.Close() +type testDDLJobIDCallback struct { + ddl.Callback + jobID int64 +} - for { - _, h, err := it.Next() - if terror.ErrorEqual(err, io.EOF) { - break - } +func (t *testDDLJobIDCallback) OnJobUpdated(job *model.Job) { + if t.jobID == 0 { + t.jobID = job.ID + } + if t.Callback != nil { + t.Callback.OnJobUpdated(job) + } +} - c.Assert(err, IsNil) - handles[h.IntValue()] = struct{}{} - } - return handles +func wrapJobIDExtCallback(oldCallback ddl.Callback) *testDDLJobIDCallback { + return &testDDLJobIDCallback{ + Callback: oldCallback, + jobID: 0, } +} - var handles map[int64]struct{} - for i := 0; i < waitForCleanDataRound; i++ { - handles = f() - if len(handles) != 0 { - time.Sleep(waitForCleanDataInterval) - } else { - break - } +func setupJobIDExtCallback(ctx sessionctx.Context) (jobExt *testDDLJobIDCallback, tearDown func()) { + dom := domain.GetDomain(ctx) + originHook := dom.DDL().GetHook() + jobIDExt := wrapJobIDExtCallback(originHook) + dom.DDL().SetHook(jobIDExt) + return jobIDExt, func() { + dom.DDL().SetHook(originHook) } - c.Assert(handles, HasLen, 0, Commentf("take time %v", time.Since(startTime))) +} + +func checkDelRangeAdded(tk *testkit.TestKit, jobID int64, elemID int64) { + query := `select sum(cnt) from + (select count(1) cnt from mysql.gc_delete_range where job_id = ? and element_id = ? union + select count(1) cnt from mysql.gc_delete_range_done where job_id = ? and element_id = ?) as gdr;` + tk.MustQuery(query, jobID, elemID, jobID, elemID).Check(testkit.Rows("1")) } func checkGlobalIndexCleanUpDone(c *C, ctx sessionctx.Context, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, pid int64) int { @@ -6416,13 +6391,15 @@ func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp m c.Assert(tb.Meta().Lock, IsNil) } } -func checkTableCache(c *C, se session.Session, dbName, tableName string) { + +func checkTableCacheStatus(c *C, se session.Session, dbName, tableName string, status model.TableCacheStatusType) { tb := testGetTableByName(c, se, dbName, tableName) dom := domain.GetDomain(se) err := dom.Reload() c.Assert(err, IsNil) - c.Assert(tb.Meta().TableCacheStatusType, Equals, model.TableCacheStatusEnable) + c.Assert(tb.Meta().TableCacheStatusType, Equals, status) } + func (s *testDBSuite2) TestDDLWithInvalidTableInfo(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -7312,18 +7289,12 @@ func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dro tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) } ctx := tk.Se.(sessionctx.Context) - t := testGetTableByName(c, ctx, "test_db", "test_drop_indexes") - var idxs []table.Index - for _, tidx := range t.Indices() { - for _, idxName := range idxNames { - if tidx.Meta().Name.L == idxName { - idxs = append(idxs, tidx) - break - } - } + idxIDs := make([]int64, 0, 3) + for _, idxName := range idxNames { + idxIDs = append(idxIDs, testGetIndexID(c, ctx, "test_db", "test_drop_indexes", idxName)) } - c.Assert(idxs, NotNil) - + jobIDExt, resetHook := setupJobIDExtCallback(ctx) + defer resetHook() testddlutil.SessionExecInGoroutine(store, dropIdxSQL, done) ticker := time.NewTicker(lease / 2) @@ -7347,23 +7318,8 @@ LOOP: num += step } } - - // Check in index, it must be no index in KV. - // Make sure there is no index with name c2_index、c3_index. - t = testGetTableByName(c, ctx, "test_db", "test_drop_indexes") - var nidxs []table.Index - for _, tidx := range t.Indices() { - for _, ids := range idxs { - if tidx.Meta().Name.L == ids.Meta().Name.L { - nidxs = append(nidxs, tidx) - } - } - } - c.Assert(nidxs, IsNil) - - for _, idx := range idxs { - idx := tables.NewIndex(t.Meta().ID, t.Meta(), idx.Meta()) - checkDelRangeDone(c, ctx, idx) + for _, idxID := range idxIDs { + checkDelRangeAdded(tk, jobIDExt.jobID, idxID) } } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 56300c607eb78..0129d7d1d8d90 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2857,6 +2857,8 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast err = d.AlterTablePartitionOptions(sctx, ident, spec) case ast.AlterTableCache: err = d.AlterTableCache(sctx, ident) + case ast.AlterTableNoCache: + err = d.AlterTableNoCache(sctx, ident) default: // Nothing to do now. } @@ -4929,6 +4931,9 @@ func (d *ddl) RenameIndex(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt if err != nil { return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) } + if tb.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Rename Index")) + } duplicate, err := validateRenameIndex(spec.FromKey, spec.ToKey, tb.Meta()) if duplicate { return nil @@ -5398,6 +5403,9 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde return errors.Trace(err) } + if t.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Create Index")) + } // Deal with anonymous index. if len(indexName.L) == 0 { colName := model.NewCIStr("expression_index") @@ -5662,6 +5670,9 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI if err != nil { return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) } + if t.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Drop Index")) + } indexInfo := t.Meta().FindIndexByName(indexName.L) @@ -5715,6 +5726,9 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt return err } + if t.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Drop Indexes")) + } indexNames := make([]model.CIStr, 0, len(specs)) ifExists := make([]bool, 0, len(specs)) for _, spec := range specs { @@ -6568,6 +6582,7 @@ func (d *ddl) AlterPlacementPolicy(ctx sessionctx.Context, stmt *ast.AlterPlacem err = d.callHookOnChanged(err) return errors.Trace(err) } + func (d *ddl) AlterTableCache(ctx sessionctx.Context, ti ast.Ident) (err error) { schema, t, err := d.getSchemaAndTableByIdent(ctx, ti) if err != nil { @@ -6598,3 +6613,27 @@ func (d *ddl) AlterTableCache(ctx sessionctx.Context, ti ast.Ident) (err error) err = d.callHookOnChanged(err) return errors.Trace(err) } + +func (d *ddl) AlterTableNoCache(ctx sessionctx.Context, ti ast.Ident) (err error) { + schema, t, err := d.getSchemaAndTableByIdent(ctx, ti) + if err != nil { + return err + } + // if a table is not in cache state, return directly + if t.Meta().TableCacheStatusType == model.TableCacheStatusDisable { + return nil + } + + job := &model.Job{ + SchemaID: schema.ID, + SchemaName: schema.Name.L, + TableID: t.Meta().ID, + Type: model.ActionAlterNoCacheTable, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{}, + } + + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 06c39207f300c..6cc3a77461c56 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -835,6 +835,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onAlterTablePlacement(d, t, job) case model.ActionAlterCacheTable: ver, err = onAlterCacheTable(t, job) + case model.ActionAlterNoCacheTable: + ver, err = onAlterNoCacheTable(t, job) default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/index.go b/ddl/index.go index 947cb9e6e2656..c93d2602a2cc2 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -304,6 +304,9 @@ func onRenameIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { if err != nil || tblInfo == nil { return ver, errors.Trace(err) } + if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { + return ver, errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Rename Index")) + } idx := tblInfo.FindIndexByName(from.L) idx.Name = to @@ -402,6 +405,9 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo if err != nil { return ver, errors.Trace(err) } + if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { + return ver, errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Create Index")) + } var ( unique bool @@ -611,6 +617,9 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { if err != nil { return ver, errors.Trace(err) } + if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { + return ver, errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Drop Index")) + } dependentHiddenCols := make([]*model.ColumnInfo, 0) for _, indexColumn := range indexInfo.Columns { @@ -732,6 +741,9 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { if err != nil { return ver, errors.Trace(err) } + if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { + return ver, errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("Drop Indexes")) + } indexInfos, err := checkDropIndexes(tblInfo, job, indexNames, ifExists) if err != nil { diff --git a/ddl/partition.go b/ddl/partition.go index a8924f5cd5e51..1cf46f6cb91e7 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -116,6 +116,25 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v job.State = model.JobStateCancelled return ver, errors.Trace(err) } + + // modify placement settings + for _, def := range addingDefinitions { + if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, def.PlacementPolicyRef); err != nil { + return ver, errors.Trace(err) + } + } + + bundles, err := alterTablePartitionBundles(t, tblInfo, addingDefinitions) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + if err = infosync.PutRuleBundles(context.TODO(), bundles); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") + } + // move the adding definition into tableInfo. updateAddingPartitionInfo(partInfo, tblInfo) ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, true) @@ -153,28 +172,6 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, errors.Trace(err) } - var bundles []*placement.Bundle - // bundle for table should be recomputed because it includes some default configs for partitions - tblBundle, err := newBundleFromTblInfo(t, job, tblInfo) - if err != nil { - return ver, errors.Trace(err) - } - - if tblBundle != nil { - bundles = append(bundles, tblBundle) - } - - partitionBundles, err := newBundlesFromPartitionDefs(t, job, addingDefinitions) - if err != nil { - return ver, errors.Trace(err) - } - bundles = append(bundles, partitionBundles...) - - if err = infosync.PutRuleBundles(context.TODO(), bundles); err != nil { - job.State = model.JobStateCancelled - return ver, errors.Wrapf(err, "failed to notify PD the placement rules") - } - // Finish this job. job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) asyncNotifyEvent(d, &util.Event{Tp: model.ActionAddTablePartition, TableInfo: tblInfo, PartInfo: partInfo}) @@ -185,6 +182,28 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, errors.Trace(err) } +func alterTablePartitionBundles(t *meta.Meta, tblInfo *model.TableInfo, addingDefinitions []model.PartitionDefinition) ([]*placement.Bundle, error) { + var bundles []*placement.Bundle + + // bundle for table should be recomputed because it includes some default configs for partitions + tblBundle, err := placement.NewTableBundle(t, tblInfo) + if err != nil { + return nil, errors.Trace(err) + } + + if tblBundle != nil { + bundles = append(bundles, tblBundle) + } + + partitionBundles, err := placement.NewPartitionListBundles(t, addingDefinitions) + if err != nil { + return nil, errors.Trace(err) + } + + bundles = append(bundles, partitionBundles...) + return bundles, nil +} + // updatePartitionInfo merge `addingDefinitions` into `Definitions` in the tableInfo. func updatePartitionInfo(tblInfo *model.TableInfo) { parInfo := &model.PartitionInfo{} @@ -1194,7 +1213,7 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e } } - bundles, err := newBundlesFromPartitionDefs(t, job, newPartitions) + bundles, err := placement.NewPartitionListBundles(t, newPartitions) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -1397,23 +1416,14 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo // the follow code is a swap function for rules of two partitions // though partitions has exchanged their ID, swap still take effect - bundles := make([]*placement.Bundle, 0, 2) - ptBundle, ptOK := d.infoCache.GetLatest().BundleByName(placement.GroupID(partDef.ID)) - ptOK = ptOK && !ptBundle.IsEmpty() - ntBundle, ntOK := d.infoCache.GetLatest().BundleByName(placement.GroupID(nt.ID)) - ntOK = ntOK && !ntBundle.IsEmpty() - if ptOK && ntOK { - bundles = append(bundles, ptBundle.Clone().Reset(placement.RuleIndexPartition, []int64{nt.ID})) - bundles = append(bundles, ntBundle.Clone().Reset(placement.RuleIndexPartition, []int64{partDef.ID})) - } else if ptOK { - bundles = append(bundles, placement.NewBundle(partDef.ID)) - bundles = append(bundles, ptBundle.Clone().Reset(placement.RuleIndexPartition, []int64{nt.ID})) - } else if ntOK { - bundles = append(bundles, placement.NewBundle(nt.ID)) - bundles = append(bundles, ntBundle.Clone().Reset(placement.RuleIndexPartition, []int64{partDef.ID})) - } - err = infosync.PutRuleBundles(context.TODO(), bundles) + + bundles, err := bundlesForExchangeTablePartition(t, job, pt, partDef, nt) if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + if err = infosync.PutRuleBundles(context.TODO(), bundles); err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } @@ -1463,6 +1473,48 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, nil } +func bundlesForExchangeTablePartition(t *meta.Meta, job *model.Job, pt *model.TableInfo, newPar *model.PartitionDefinition, nt *model.TableInfo) ([]*placement.Bundle, error) { + bundles := make([]*placement.Bundle, 0, 3) + + ptBundle, err := placement.NewTableBundle(t, pt) + if err != nil { + return nil, errors.Trace(err) + } + if ptBundle != nil { + bundles = append(bundles, ptBundle) + } + + parBundle, err := placement.NewPartitionBundle(t, *newPar) + if err != nil { + return nil, errors.Trace(err) + } + if parBundle != nil { + bundles = append(bundles, parBundle) + } + + ntBundle, err := placement.NewTableBundle(t, nt) + if err != nil { + return nil, errors.Trace(err) + } + if ntBundle != nil { + bundles = append(bundles, ntBundle) + } + + if parBundle == nil && ntBundle != nil { + // newPar.ID is the ID of old table to exchange, so ntBundle != nil means it has some old placement settings. + // We should remove it in this situation + bundles = append(bundles, placement.NewBundle(newPar.ID)) + } + + if parBundle != nil && ntBundle == nil { + // nt.ID is the ID of old partition to exchange, so parBundle != nil means it has some old placement settings. + // We should remove it in this situation + bundles = append(bundles, placement.NewBundle(nt.ID)) + } + + return bundles, nil +} + func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, index int, schemaName, tableName model.CIStr) error { var sql string var paramList []interface{} diff --git a/ddl/placement/bundle.go b/ddl/placement/bundle.go index 85de67845de00..9c493dccdf619 100644 --- a/ddl/placement/bundle.go +++ b/ddl/placement/bundle.go @@ -25,6 +25,7 @@ import ( "strings" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" @@ -413,3 +414,99 @@ func (b *Bundle) GetLeaderDC(dcLabelKey string) (string, bool) { } return "", false } + +// NewTableBundle creates a bundle for table key range. +// If table is a partitioned table, it also contains the rules that inherited from table for every partition. +// The bundle does not contain the rules specified independently by each partition +func NewTableBundle(t *meta.Meta, tbInfo *model.TableInfo) (*Bundle, error) { + bundle, err := newBundleFromPolicyOrDirectOptions(t, tbInfo.PlacementPolicyRef, tbInfo.DirectPlacementOpts) + if err != nil { + return nil, err + } + + if bundle == nil { + return nil, nil + } + ids := []int64{tbInfo.ID} + // build the default partition rules in the table-level bundle. + if tbInfo.Partition != nil { + for _, pDef := range tbInfo.Partition.Definitions { + ids = append(ids, pDef.ID) + } + } + bundle.Reset(RuleIndexTable, ids) + return bundle, nil +} + +// NewPartitionBundle creates a bundle for partition key range. +// It only contains the rules specified independently by the partition. +// That is to say the inherited rules from table is not included. +func NewPartitionBundle(t *meta.Meta, def model.PartitionDefinition) (*Bundle, error) { + bundle, err := newBundleFromPolicyOrDirectOptions(t, def.PlacementPolicyRef, def.DirectPlacementOpts) + if err != nil { + return nil, err + } + + if bundle != nil { + bundle.Reset(RuleIndexPartition, []int64{def.ID}) + } + + return bundle, nil +} + +// NewPartitionListBundles creates a bundle list for a partition list +func NewPartitionListBundles(t *meta.Meta, defs []model.PartitionDefinition) ([]*Bundle, error) { + bundles := make([]*Bundle, 0, len(defs)) + // If the partition has the placement rules on their own, build the partition-level bundles additionally. + for _, def := range defs { + bundle, err := NewPartitionBundle(t, def) + if err != nil { + return nil, err + } + + if bundle != nil { + bundles = append(bundles, bundle) + } + } + return bundles, nil +} + +// NewFullTableBundles returns a bundle list with both table bundle and partition bundles +func NewFullTableBundles(t *meta.Meta, tbInfo *model.TableInfo) ([]*Bundle, error) { + var bundles []*Bundle + tableBundle, err := NewTableBundle(t, tbInfo) + if err != nil { + return nil, err + } + + if tableBundle != nil { + bundles = append(bundles, tableBundle) + } + + if tbInfo.Partition != nil { + partitionBundles, err := NewPartitionListBundles(t, tbInfo.Partition.Definitions) + if err != nil { + return nil, err + } + bundles = append(bundles, partitionBundles...) + } + + return bundles, nil +} + +func newBundleFromPolicyOrDirectOptions(t *meta.Meta, ref *model.PolicyRefInfo, directOpts *model.PlacementSettings) (*Bundle, error) { + if directOpts != nil { + return NewBundleFromOptions(directOpts) + } + + if ref != nil { + policy, err := t.GetPolicy(ref.ID) + if err != nil { + return nil, err + } + + return NewBundleFromOptions(policy.PlacementSettings) + } + + return nil, nil +} diff --git a/ddl/placement/meta_bundle_test.go b/ddl/placement/meta_bundle_test.go new file mode 100644 index 0000000000000..0306458c1a745 --- /dev/null +++ b/ddl/placement/meta_bundle_test.go @@ -0,0 +1,379 @@ +// 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 placement_test + +import ( + "context" + "encoding/hex" + "encoding/json" + "fmt" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/ddl/placement" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/util/codec" +) + +var _ = Suite(&testMetaBundleSuite{}) + +type testMetaBundleSuite struct { + policy1 *model.PolicyInfo + policy2 *model.PolicyInfo + tbl1 *model.TableInfo + tbl2 *model.TableInfo + tbl3 *model.TableInfo + tbl4 *model.TableInfo +} + +func (s *testMetaBundleSuite) SetUpSuite(c *C) { + s.policy1 = &model.PolicyInfo{ + ID: 11, + Name: model.NewCIStr("p1"), + PlacementSettings: &model.PlacementSettings{ + PrimaryRegion: "r1", + Regions: "r1,r2", + }, + State: model.StatePublic, + } + s.policy2 = &model.PolicyInfo{ + ID: 12, + Name: model.NewCIStr("p2"), + PlacementSettings: &model.PlacementSettings{ + PrimaryRegion: "r2", + Regions: "r1,r2", + }, + State: model.StatePublic, + } + s.tbl1 = &model.TableInfo{ + ID: 101, + Name: model.NewCIStr("t1"), + PlacementPolicyRef: &model.PolicyRefInfo{ + ID: 11, + Name: model.NewCIStr("p1"), + }, + Partition: &model.PartitionInfo{ + Definitions: []model.PartitionDefinition{ + { + ID: 1000, + Name: model.NewCIStr("par0"), + }, + { + ID: 1001, + Name: model.NewCIStr("par1"), + PlacementPolicyRef: &model.PolicyRefInfo{ID: 12, Name: model.NewCIStr("p2")}, + }, + { + ID: 1002, + Name: model.NewCIStr("par2"), + }, + { + ID: 1003, + Name: model.NewCIStr("par3"), + DirectPlacementOpts: &model.PlacementSettings{PrimaryRegion: "r3", Regions: "r3"}, + }, + }, + }, + } + s.tbl2 = &model.TableInfo{ + ID: 102, + Name: model.NewCIStr("t2"), + Partition: &model.PartitionInfo{ + Definitions: []model.PartitionDefinition{ + { + ID: 1000, + Name: model.NewCIStr("par0"), + PlacementPolicyRef: &model.PolicyRefInfo{ID: 11, Name: model.NewCIStr("p1")}, + }, + { + ID: 1001, + Name: model.NewCIStr("par1"), + }, + { + ID: 1002, + Name: model.NewCIStr("par2"), + DirectPlacementOpts: &model.PlacementSettings{PrimaryRegion: "r2", Regions: "r2"}, + }, + { + ID: 1003, + Name: model.NewCIStr("par3"), + }, + }, + }, + } + s.tbl3 = &model.TableInfo{ + ID: 103, + Name: model.NewCIStr("t3"), + DirectPlacementOpts: &model.PlacementSettings{ + LeaderConstraints: "[+region=bj]", + }, + } + s.tbl4 = &model.TableInfo{ + ID: 104, + Name: model.NewCIStr("t4"), + } +} + +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) + return nil + }), IsNil) +} + +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) + + // tbl1 + bundle, err := placement.NewTableBundle(t, s.tbl1) + c.Assert(err, IsNil) + s.checkTableBundle(c, s.tbl1, bundle) + + // tbl2 + bundle, err = placement.NewTableBundle(t, s.tbl2) + c.Assert(err, IsNil) + s.checkTableBundle(c, s.tbl2, bundle) + + // tbl3 + bundle, err = placement.NewTableBundle(t, s.tbl3) + c.Assert(err, IsNil) + s.checkTableBundle(c, s.tbl3, bundle) + + // tbl4 + bundle, err = placement.NewTableBundle(t, s.tbl4) + c.Assert(err, IsNil) + s.checkTableBundle(c, 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) + + c.Assert(kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { + t := 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) + + // 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) + + // tbl1.par3 + bundle, err = placement.NewPartitionBundle(t, s.tbl1.Partition.Definitions[3]) + c.Assert(err, IsNil) + s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[3], bundle) + + return nil + }), IsNil) +} + +func (s *testMetaBundleSuite) TestNewPartitionListBundles(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) + + bundles, err := placement.NewPartitionListBundles(t, s.tbl1.Partition.Definitions) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 2) + s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[1], bundles[0]) + s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[3], bundles[1]) + + bundles, err = placement.NewPartitionListBundles(t, []model.PartitionDefinition{}) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 0) + + bundles, err = placement.NewPartitionListBundles(t, []model.PartitionDefinition{ + s.tbl1.Partition.Definitions[0], + s.tbl1.Partition.Definitions[2], + }) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 0) + + return nil + }), IsNil) +} + +func (s *testMetaBundleSuite) TestNewFullTableBundles(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) + + bundles, err := placement.NewFullTableBundles(t, s.tbl1) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 3) + s.checkTableBundle(c, s.tbl1, bundles[0]) + s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[1], bundles[1]) + s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[3], bundles[2]) + + bundles, err = placement.NewFullTableBundles(t, s.tbl2) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 2) + s.checkPartitionBundle(c, s.tbl2.Partition.Definitions[0], bundles[0]) + s.checkPartitionBundle(c, s.tbl2.Partition.Definitions[2], bundles[1]) + + 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(t, s.tbl4) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 0) + + return nil + }), IsNil) +} + +func (s *testMetaBundleSuite) checkTwoJSONObjectEquals(c *C, expected interface{}, got interface{}) { + expectedJSON, err := json.Marshal(expected) + c.Assert(err, IsNil) + expectedStr := string(expectedJSON) + + gotJSON, err := json.Marshal(got) + c.Assert(err, IsNil) + gotStr := string(gotJSON) + + c.Assert(gotStr, Equals, expectedStr) +} + +func (s *testMetaBundleSuite) checkTableBundle(c *C, tbl *model.TableInfo, got *placement.Bundle) { + if tbl.PlacementPolicyRef == nil && tbl.DirectPlacementOpts == nil { + c.Assert(got, IsNil) + return + } + + expected := &placement.Bundle{ + ID: fmt.Sprintf("TiDB_DDL_%d", tbl.ID), + Index: placement.RuleIndexTable, + Override: true, + Rules: s.expectedRules(c, tbl.PlacementPolicyRef, tbl.DirectPlacementOpts), + } + + for idx, rule := range expected.Rules { + rule.GroupID = expected.ID + rule.Index = placement.RuleIndexTable + rule.ID = fmt.Sprintf("table_rule_%d_%d", tbl.ID, idx) + rule.StartKeyHex = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(tbl.ID))) + rule.EndKeyHex = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(tbl.ID+1))) + } + + if tbl.Partition != nil { + for _, par := range tbl.Partition.Definitions { + rules := s.expectedRules(c, tbl.PlacementPolicyRef, tbl.DirectPlacementOpts) + for idx, rule := range rules { + rule.GroupID = expected.ID + rule.Index = placement.RuleIndexPartition + rule.ID = fmt.Sprintf("partition_rule_%d_%d", par.ID, idx) + rule.StartKeyHex = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(par.ID))) + rule.EndKeyHex = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(par.ID+1))) + expected.Rules = append(expected.Rules, rule) + } + } + } + + s.checkTwoJSONObjectEquals(c, expected, got) +} + +func (s *testMetaBundleSuite) checkPartitionBundle(c *C, def model.PartitionDefinition, got *placement.Bundle) { + if def.PlacementPolicyRef == nil && def.DirectPlacementOpts == nil { + c.Assert(got, IsNil) + return + } + + expected := &placement.Bundle{ + ID: fmt.Sprintf("TiDB_DDL_%d", def.ID), + Index: placement.RuleIndexPartition, + Override: true, + Rules: s.expectedRules(c, def.PlacementPolicyRef, def.DirectPlacementOpts), + } + + for idx, rule := range expected.Rules { + rule.GroupID = expected.ID + rule.Index = placement.RuleIndexTable + rule.ID = fmt.Sprintf("partition_rule_%d_%d", def.ID, idx) + rule.StartKeyHex = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(def.ID))) + rule.EndKeyHex = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(def.ID+1))) + } + + s.checkTwoJSONObjectEquals(c, expected, got) +} + +func (s *testMetaBundleSuite) expectedRules(c *C, ref *model.PolicyRefInfo, direct *model.PlacementSettings) []*placement.Rule { + c.Assert(ref != nil && direct != nil, IsFalse) + + var settings *model.PlacementSettings + if ref != nil { + var policy *model.PolicyInfo + switch ref.ID { + case s.policy1.ID: + policy = s.policy1 + case s.policy2.ID: + policy = s.policy2 + default: + c.FailNow() + } + c.Assert(ref.Name, Equals, policy.Name) + settings = policy.PlacementSettings + } + + if direct != nil { + settings = direct + } + + if settings == nil { + return []*placement.Rule{} + } + + bundle, err := placement.NewBundleFromOptions(settings) + c.Assert(err, IsNil) + return bundle.Rules +} + +func newMockStore(c *C) kv.Storage { + store, err := mockstore.NewMockStore() + c.Assert(err, IsNil) + return store +} diff --git a/ddl/placement_policy.go b/ddl/placement_policy.go index 1dec5315caa9e..0fcd17270c1ab 100644 --- a/ddl/placement_policy.go +++ b/ddl/placement_policy.go @@ -124,6 +124,32 @@ func checkPlacementPolicyExistAndCancelNonExistJob(t *meta.Meta, job *model.Job, return nil, err } +func checkPlacementPolicyRefValidAndCanNonValidJob(t *meta.Meta, job *model.Job, ref *model.PolicyRefInfo) (*model.PolicyInfo, error) { + if ref == nil { + return nil, nil + } + + return checkPlacementPolicyExistAndCancelNonExistJob(t, job, ref.ID) +} + +func checkAllTablePlacementPoliciesExistAndCancelNonExistJob(t *meta.Meta, job *model.Job, tblInfo *model.TableInfo) error { + if _, err := checkPlacementPolicyRefValidAndCanNonValidJob(t, job, tblInfo.PlacementPolicyRef); err != nil { + return errors.Trace(err) + } + + if tblInfo.Partition == nil { + return nil + } + + for _, def := range tblInfo.Partition.Definitions { + if _, err := checkPlacementPolicyRefValidAndCanNonValidJob(t, job, def.PlacementPolicyRef); err != nil { + return errors.Trace(err) + } + } + + return nil +} + func onDropPlacementPolicy(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { policyInfo, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, job.SchemaID) if err != nil { diff --git a/ddl/placement_policy_test.go b/ddl/placement_policy_test.go index 35e28bf0a2b71..65b5e7994f713 100644 --- a/ddl/placement_policy_test.go +++ b/ddl/placement_policy_test.go @@ -1386,3 +1386,161 @@ func (s *testDBSuite6) TestTruncateTablePartitionWithPlacement(c *C) { " PARTITION `p3` VALUES LESS THAN (100000) /*T![placement] PRIMARY_REGION=\"r2\" REGIONS=\"r2\" */\n" + ")")) } + +func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_enable_exchange_partition=1") + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, tp") + tk.MustExec("drop placement policy if exists p1") + tk.MustExec("drop placement policy if exists p2") + + tk.MustExec("create placement policy p1 primary_region='r1' regions='r1'") + defer tk.MustExec("drop placement policy p1") + + tk.MustExec("create placement policy p2 primary_region='r2' regions='r2'") + defer tk.MustExec("drop placement policy p2") + + policy1, ok := tk.Se.GetInfoSchema().(infoschema.InfoSchema).PolicyByName(model.NewCIStr("p1")) + c.Assert(ok, IsTrue) + + policy2, ok := tk.Se.GetInfoSchema().(infoschema.InfoSchema).PolicyByName(model.NewCIStr("p2")) + c.Assert(ok, IsTrue) + + tk.MustExec(`CREATE TABLE t1 (id INT) placement policy p1`) + defer tk.MustExec("drop table t1") + + tk.MustExec(`CREATE TABLE t2 (id INT)`) + defer tk.MustExec("drop table t2") + + t1, err := tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + t1ID := t1.Meta().ID + + t2, err := tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + c.Assert(err, IsNil) + t2ID := t2.Meta().ID + + tk.MustExec(`CREATE TABLE tp (id INT) primary_region="r1" regions="r1" PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100), + PARTITION p1 VALUES LESS THAN (1000) placement policy p2, + PARTITION p2 VALUES LESS THAN (10000) primary_region="r1" regions="r1,r2" + );`) + defer tk.MustExec("drop table tp") + + tp, err := tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + c.Assert(err, IsNil) + tpID := tp.Meta().ID + par0ID := tp.Meta().Partition.Definitions[0].ID + par1ID := tp.Meta().Partition.Definitions[1].ID + par2ID := tp.Meta().Partition.Definitions[2].ID + + // exchange par0, t1 + tk.MustExec("alter table tp exchange partition p0 with table t1") + tk.MustQuery("show create table t1").Check(testkit.Rows("" + + "t1 CREATE TABLE `t1` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */")) + tk.MustQuery("show create table tp").Check(testkit.Rows("" + + "tp CREATE TABLE `tp` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1\" */\n" + + "PARTITION BY RANGE ( `id` ) (\n" + + " PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p2` */,\n" + + " PARTITION `p2` VALUES LESS THAN (10000) /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" */\n" + + ")")) + tp, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + c.Assert(err, IsNil) + c.Assert(tp.Meta().ID, Equals, tpID) + c.Assert(tp.Meta().Partition.Definitions[0].ID, Equals, t1ID) + c.Assert(tp.Meta().Partition.Definitions[0].DirectPlacementOpts, IsNil) + c.Assert(tp.Meta().Partition.Definitions[0].PlacementPolicyRef, IsNil) + t1, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + c.Assert(t1.Meta().ID, Equals, par0ID) + c.Assert(t1.Meta().DirectPlacementOpts, IsNil) + c.Assert(t1.Meta().PlacementPolicyRef.ID, Equals, policy1.ID) + + // exchange par0, t2 + tk.MustExec("alter table tp exchange partition p0 with table t2") + 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")) + tk.MustQuery("show create table tp").Check(testkit.Rows("" + + "tp CREATE TABLE `tp` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1\" */\n" + + "PARTITION BY RANGE ( `id` ) (\n" + + " PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p2` */,\n" + + " PARTITION `p2` VALUES LESS THAN (10000) /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" */\n" + + ")")) + tp, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + c.Assert(err, IsNil) + c.Assert(tp.Meta().ID, Equals, tpID) + c.Assert(tp.Meta().Partition.Definitions[0].ID, Equals, t2ID) + c.Assert(tp.Meta().Partition.Definitions[0].DirectPlacementOpts, IsNil) + c.Assert(tp.Meta().Partition.Definitions[0].PlacementPolicyRef, IsNil) + t2, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + c.Assert(err, IsNil) + c.Assert(t2.Meta().ID, Equals, t1ID) + c.Assert(t2.Meta().DirectPlacementOpts, IsNil) + c.Assert(t2.Meta().PlacementPolicyRef, IsNil) + + // exchange par1, t1 + tk.MustExec("alter table tp exchange partition p1 with table t1") + tk.MustQuery("show create table t1").Check(testkit.Rows("" + + "t1 CREATE TABLE `t1` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */")) + tk.MustQuery("show create table tp").Check(testkit.Rows("" + + "tp CREATE TABLE `tp` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1\" */\n" + + "PARTITION BY RANGE ( `id` ) (\n" + + " PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p2` */,\n" + + " PARTITION `p2` VALUES LESS THAN (10000) /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" */\n" + + ")")) + tp, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + c.Assert(err, IsNil) + c.Assert(tp.Meta().ID, Equals, tpID) + c.Assert(tp.Meta().Partition.Definitions[1].ID, Equals, par0ID) + c.Assert(tp.Meta().Partition.Definitions[1].DirectPlacementOpts, IsNil) + c.Assert(tp.Meta().Partition.Definitions[1].PlacementPolicyRef.ID, Equals, policy2.ID) + t1, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + c.Assert(t1.Meta().ID, Equals, par1ID) + c.Assert(t1.Meta().DirectPlacementOpts, IsNil) + c.Assert(t1.Meta().PlacementPolicyRef.ID, Equals, policy1.ID) + + // exchange par2, t2 + tk.MustExec("alter table tp exchange partition p2 with table t2") + 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")) + tk.MustQuery("show create table tp").Check(testkit.Rows("" + + "tp CREATE TABLE `tp` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1\" */\n" + + "PARTITION BY RANGE ( `id` ) (\n" + + " PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p2` */,\n" + + " PARTITION `p2` VALUES LESS THAN (10000) /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" */\n" + + ")")) + tp, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + c.Assert(err, IsNil) + c.Assert(tp.Meta().ID, Equals, tpID) + c.Assert(tp.Meta().Partition.Definitions[2].ID, Equals, t1ID) + c.Assert(tp.Meta().Partition.Definitions[2].DirectPlacementOpts.PrimaryRegion, Equals, "r1") + c.Assert(tp.Meta().Partition.Definitions[2].DirectPlacementOpts.Regions, Equals, "r1,r2") + c.Assert(tp.Meta().Partition.Definitions[2].PlacementPolicyRef, IsNil) + t2, err = tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + c.Assert(err, IsNil) + c.Assert(t2.Meta().ID, Equals, par2ID) + c.Assert(t2.Meta().DirectPlacementOpts, IsNil) + c.Assert(t2.Meta().PlacementPolicyRef, IsNil) +} diff --git a/ddl/schema.go b/ddl/schema.go index d850c2fc61410..718788cc8a524 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -157,11 +157,8 @@ func onModifySchemaDefaultPlacement(t *meta.Meta, job *model.Job) (ver int64, _ return ver, errors.Trace(err) } // Double Check if policy exits while ddl executing - if placementPolicyRef != nil { - _, err = checkPlacementPolicyExistAndCancelNonExistJob(t, job, placementPolicyRef.ID) - if err != nil { - return ver, errors.Trace(err) - } + if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, placementPolicyRef); err != nil { + return ver, errors.Trace(err) } // Notice: dbInfo.DirectPlacementOpts and dbInfo.PlacementPolicyRef can not be both not nil, which checked before constructing ddl job. diff --git a/ddl/table.go b/ddl/table.go index 191bf895eaab8..4a4b2fb09d0f9 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -68,13 +68,6 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) return ver, errors.Trace(err) } - // build table & partition bundles if any. - bundles, err := fullBundlesFromTblInfo(t, job, tbInfo) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - ver, err = updateSchemaVersion(t, job) if err != nil { return ver, errors.Trace(err) @@ -95,6 +88,18 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) failpoint.Return(ver, errors.New("mock create table error")) } }) + + // build table & partition bundles if any. + if err = checkAllTablePlacementPoliciesExistAndCancelNonExistJob(t, job, tbInfo); err != nil { + return ver, errors.Trace(err) + } + + bundles, err := placement.NewFullTableBundles(t, tbInfo) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + // Send the placement bundle to PD. err = infosync.PutRuleBundles(context.TODO(), bundles) if err != nil { @@ -111,105 +116,6 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) } } -// fullBundlesFromTblInfo returns a bundle list with both table bundle and partition bundles -func fullBundlesFromTblInfo(t *meta.Meta, job *model.Job, tbInfo *model.TableInfo) ([]*placement.Bundle, error) { - var bundles []*placement.Bundle - tableBundle, err := newBundleFromTblInfo(t, job, tbInfo) - if err != nil { - return nil, errors.Trace(err) - } - - if tableBundle != nil { - bundles = append(bundles, tableBundle) - } - - if tbInfo.Partition != nil { - partitionBundles, err := newBundlesFromPartitionDefs(t, job, tbInfo.Partition.Definitions) - if err != nil { - return nil, errors.Trace(err) - } - bundles = append(bundles, partitionBundles...) - } - - return bundles, nil -} - -func newBundleFromTblInfo(t *meta.Meta, job *model.Job, tbInfo *model.TableInfo) (*placement.Bundle, error) { - bundle, err := newBundleFromPolicyOrDirectOptions(t, job, tbInfo.PlacementPolicyRef, tbInfo.DirectPlacementOpts) - if err != nil { - return nil, errors.Trace(err) - } - if bundle == nil { - return nil, nil - } - ids := []int64{tbInfo.ID} - // build the default partition rules in the table-level bundle. - if tbInfo.Partition != nil { - for _, pDef := range tbInfo.Partition.Definitions { - ids = append(ids, pDef.ID) - } - } - bundle.Reset(placement.RuleIndexTable, ids) - return bundle, nil -} - -func newBundleFromPartitionDef(t *meta.Meta, job *model.Job, def model.PartitionDefinition) (*placement.Bundle, error) { - bundle, err := newBundleFromPolicyOrDirectOptions(t, job, def.PlacementPolicyRef, def.DirectPlacementOpts) - if err != nil { - return nil, errors.Trace(err) - } - - if bundle != nil { - bundle.Reset(placement.RuleIndexPartition, []int64{def.ID}) - } - - return bundle, nil -} - -func newBundlesFromPartitionDefs(t *meta.Meta, job *model.Job, defs []model.PartitionDefinition) ([]*placement.Bundle, error) { - bundles := make([]*placement.Bundle, 0, len(defs)) - // If the partition has the placement rules on their own, build the partition-level bundles additionally. - for _, def := range defs { - bundle, err := newBundleFromPartitionDef(t, job, def) - if err != nil { - return nil, errors.Trace(err) - } - - if bundle != nil { - bundles = append(bundles, bundle) - } - } - return bundles, nil -} - -func newBundleFromPolicyOrDirectOptions(t *meta.Meta, job *model.Job, ref *model.PolicyRefInfo, directOpts *model.PlacementSettings) (*placement.Bundle, error) { - if directOpts != nil { - // build bundle from direct placement options. - bundle, err := placement.NewBundleFromOptions(directOpts) - if err != nil { - job.State = model.JobStateCancelled - return nil, errors.Trace(err) - } - return bundle, nil - } - if ref != nil { - // placement policy reference will override the direct placement options. - po, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, ref.ID) - if err != nil { - job.State = model.JobStateCancelled - return nil, errors.Trace(infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs(ref.Name)) - } - // build bundle from placement policy. - bundle, err := placement.NewBundleFromOptions(po.PlacementSettings) - if err != nil { - job.State = model.JobStateCancelled - return nil, errors.Trace(err) - } - return bundle, nil - } - return nil, nil -} - func createTableOrViewWithCheck(t *meta.Meta, job *model.Job, schemaID int64, tbInfo *model.TableInfo) error { err := checkTableInfoValid(tbInfo) if err != nil { @@ -668,7 +574,7 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro tblInfo.ID = newTableID // build table & partition bundles if any. - bundles, err := fullBundlesFromTblInfo(t, job, tblInfo) + bundles, err := placement.NewFullTableBundles(t, tblInfo) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -1377,8 +1283,13 @@ func onAlterTablePartitionOptions(d *ddlCtx, t *meta.Meta, job *model.Job) (ver return ver, errors.Trace(err) } - bundle, err := newBundleFromPartitionDef(t, job, *partitionDef) + if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, partitionDef.PlacementPolicyRef); err != nil { + return ver, errors.Trace(err) + } + + bundle, err := placement.NewPartitionBundle(t, *partitionDef) if err != nil { + job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -1415,9 +1326,14 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, tblInfo.PlacementPolicyRef = policyRefInfo tblInfo.DirectPlacementOpts = placementSettings - bundle, err := newBundleFromTblInfo(t, job, tblInfo) + if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, policyRefInfo); err != nil { + return 0, errors.Trace(err) + } + + bundle, err := placement.NewTableBundle(t, tblInfo) if err != nil { - return 0, err + job.State = model.JobStateCancelled + return 0, errors.Trace(err) } ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) @@ -1432,6 +1348,11 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err = dropRuleBundles(d, []int64{tblInfo.ID}) } + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) return ver, nil @@ -1477,6 +1398,7 @@ func updateLabelRules(job *model.Job, tblInfo *model.TableInfo, oldRules map[str patch := label.NewRulePatch(newRules, oldRuleIDs) return infosync.UpdateLabelRules(context.TODO(), patch) } + func onAlterCacheTable(t *meta.Meta, job *model.Job) (ver int64, err error) { tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { @@ -1519,3 +1441,38 @@ func onAlterCacheTable(t *meta.Meta, job *model.Job) (ver int64, err error) { } return ver, err } + +func onAlterNoCacheTable(t *meta.Meta, job *model.Job) (ver int64, err error) { + tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return 0, errors.Trace(err) + } + // If the table is not in the cache state + if tbInfo.TableCacheStatusType == model.TableCacheStatusDisable { + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + return ver, nil + } + + switch tbInfo.TableCacheStatusType { + case model.TableCacheStatusEnable: + // enable -> switching + tbInfo.TableCacheStatusType = model.TableCacheStatusSwitching + ver, err = updateVersionAndTableInfoWithCheck(t, job, tbInfo, true) + if err != nil { + return ver, err + } + case model.TableCacheStatusSwitching: + // switching -> disable + tbInfo.TableCacheStatusType = model.TableCacheStatusDisable + ver, err = updateVersionAndTableInfoWithCheck(t, job, tbInfo, true) + if err != nil { + return ver, err + } + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + default: + job.State = model.JobStateCancelled + err = ErrInvalidDDLState.GenWithStackByArgs("alter table no cache", tbInfo.TableCacheStatusType.String()) + } + return ver, err +} diff --git a/ddl/table_test.go b/ddl/table_test.go index 02bf8c41c6887..7f577c8d1a916 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -417,6 +417,11 @@ func (s *testTableSuite) TestTable(c *C) { testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic) testCheckJobDone(c, d, job, true) checkTableCacheTest(c, d, dbInfo1, tblInfo) + // for alter no cache table + job = testAlterNoCacheTable(c, ctx, d, dbInfo1.ID, tblInfo) + testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic) + testCheckJobDone(c, d, job, true) + checkTableNoCacheTest(c, d, dbInfo1, tblInfo) } func checkTableCacheTest(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) { @@ -432,6 +437,18 @@ func checkTableCacheTest(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.Tabl c.Assert(err, IsNil) } +func checkTableNoCacheTest(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) { + err := kv.RunInNewTxn(context.Background(), d.store, false, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + info, err := t.GetTable(dbInfo.ID, tblInfo.ID) + c.Assert(err, IsNil) + c.Assert(info, NotNil) + c.Assert(info.TableCacheStatusType, Equals, model.TableCacheStatusDisable) + return nil + }) + c.Assert(err, IsNil) +} + func testAlterCacheTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblInfo *model.TableInfo) *model.Job { job := &model.Job{ @@ -449,6 +466,23 @@ func testAlterCacheTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64 return job } +func testAlterNoCacheTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblInfo *model.TableInfo) *model.Job { + + job := &model.Job{ + SchemaID: newSchemaID, + TableID: tblInfo.ID, + Type: model.ActionAlterNoCacheTable, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{}, + } + err := d.doDDLJob(ctx, job) + c.Assert(err, IsNil) + + v := getSchemaVer(c, ctx) + checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v}) + return job +} + // for drop indexes func createTestTableForDropIndexes(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, name string, num int) *model.TableInfo { tableInfo := testTableInfo(c, d, name, num) diff --git a/domain/domain.go b/domain/domain.go index e77324fa37e04..8ed2c18e58cd4 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1446,13 +1446,9 @@ const ( // NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches // the key will get notification. func (do *Domain) NotifyUpdatePrivilege() error { - // If skip-grant-table is configured, do not flush privileges. - // Because LoadPrivilegeLoop does not run and the privilege Handle is nil, - // the call to do.PrivilegeHandle().Update would panic. - if config.GetGlobalConfig().Security.SkipGrantTable { - return nil - } - + // No matter skip-grant-table is configured or not, sending an etcd message is required. + // Because we need to tell other TiDB instances to update privilege data, say, we're changing the + // password using a special TiDB instance and want the new password to take effect. if do.etcdClient != nil { row := do.etcdClient.KV _, err := row.Put(context.Background(), privilegeKey, "") @@ -1460,6 +1456,14 @@ func (do *Domain) NotifyUpdatePrivilege() error { logutil.BgLogger().Warn("notify update privilege failed", zap.Error(err)) } } + + // If skip-grant-table is configured, do not flush privileges. + // Because LoadPrivilegeLoop does not run and the privilege Handle is nil, + // the call to do.PrivilegeHandle().Update would panic. + if config.GetGlobalConfig().Security.SkipGrantTable { + return nil + } + // update locally sysSessionPool := do.SysSessionPool() ctx, err := sysSessionPool.Get() diff --git a/domain/infosync/info.go b/domain/infosync/info.go index 57f7e336a4ccf..b159100f0f92e 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -15,7 +15,6 @@ package infosync import ( - "bytes" "context" "encoding/hex" "encoding/json" @@ -99,6 +98,7 @@ type InfoSyncer struct { prometheusAddr string modifyTime time.Time labelRuleManager LabelRuleManager + placementManager PlacementManager } // ServerInfo is server static information. @@ -179,8 +179,10 @@ func GlobalInfoSyncerInit(ctx context.Context, id string, serverIDGetter func() } if etcdCli != nil { is.labelRuleManager = initLabelRuleManager(etcdCli.Endpoints()) + is.placementManager = initPlacementManager(etcdCli.Endpoints()) } else { is.labelRuleManager = initLabelRuleManager([]string{}) + is.placementManager = initPlacementManager([]string{}) } setGlobalInfoSyncer(is) return is, nil @@ -215,6 +217,13 @@ func initLabelRuleManager(addrs []string) LabelRuleManager { return &PDLabelManager{addrs: addrs} } +func initPlacementManager(addrs []string) PlacementManager { + if len(addrs) == 0 { + return &mockPlacementManager{} + } + return &PDPlacementManager{addrs: addrs} +} + // GetServerInfo gets self server static information. func GetServerInfo() (*ServerInfo, error) { is, err := getGlobalInfoSyncer() @@ -401,22 +410,7 @@ func GetAllRuleBundles(ctx context.Context) ([]*placement.Bundle, error) { return nil, err } - bundles := []*placement.Bundle{} - if is.etcdCli == nil { - return bundles, nil - } - - addrs := is.etcdCli.Endpoints() - - if len(addrs) == 0 { - return nil, errors.Errorf("pd unavailable") - } - - res, err := doRequest(ctx, addrs, path.Join(pdapi.Config, "placement-rule"), "GET", nil) - if err == nil && res != nil { - err = json.Unmarshal(res, &bundles) - } - return bundles, err + return is.placementManager.GetAllRuleBundles(ctx) } // GetRuleBundle is used to get one specific rule bundle from PD. @@ -426,53 +420,17 @@ func GetRuleBundle(ctx context.Context, name string) (*placement.Bundle, error) return nil, err } - bundle := &placement.Bundle{ID: name} - - if is.etcdCli == nil { - return bundle, nil - } - - addrs := is.etcdCli.Endpoints() - - if len(addrs) == 0 { - return nil, errors.Errorf("pd unavailable") - } - - res, err := doRequest(ctx, addrs, path.Join(pdapi.Config, "placement-rule", name), "GET", nil) - if err == nil && res != nil { - err = json.Unmarshal(res, bundle) - } - return bundle, err + return is.placementManager.GetRuleBundle(ctx, name) } // PutRuleBundles is used to post specific rule bundles to PD. func PutRuleBundles(ctx context.Context, bundles []*placement.Bundle) error { - if len(bundles) == 0 { - return nil - } - is, err := getGlobalInfoSyncer() if err != nil { return err } - if is.etcdCli == nil { - return nil - } - - addrs := is.etcdCli.Endpoints() - - if len(addrs) == 0 { - return errors.Errorf("pd unavailable") - } - - b, err := json.Marshal(bundles) - if err != nil { - return err - } - - _, err = doRequest(ctx, addrs, path.Join(pdapi.Config, "placement-rule")+"?partial=true", "POST", bytes.NewReader(b)) - return err + return is.placementManager.PutRuleBundles(ctx, bundles) } func (is *InfoSyncer) getAllServerInfo(ctx context.Context) (map[string]*ServerInfo, error) { diff --git a/domain/infosync/placement_manager.go b/domain/infosync/placement_manager.go new file mode 100644 index 0000000000000..7c4db7dcd61e3 --- /dev/null +++ b/domain/infosync/placement_manager.go @@ -0,0 +1,122 @@ +// 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 infosync + +import ( + "bytes" + "context" + "encoding/json" + "path" + "sync" + + "github.com/pingcap/tidb/ddl/placement" + "github.com/pingcap/tidb/util/pdapi" +) + +// PlacementManager manages placement settings +type PlacementManager interface { + // GetRuleBundle is used to get one specific rule bundle from PD. + GetRuleBundle(ctx context.Context, name string) (*placement.Bundle, error) + // GetAllRuleBundles is used to get all rule bundles from PD. It is used to load full rules from PD while fullload infoschema. + GetAllRuleBundles(ctx context.Context) ([]*placement.Bundle, error) + // PutRuleBundles is used to post specific rule bundles to PD. + PutRuleBundles(ctx context.Context, bundles []*placement.Bundle) error +} + +// PDPlacementManager manages placement with pd +type PDPlacementManager struct { + addrs []string +} + +// GetRuleBundle is used to get one specific rule bundle from PD. +func (m *PDPlacementManager) GetRuleBundle(ctx context.Context, name string) (*placement.Bundle, error) { + bundle := &placement.Bundle{ID: name} + res, err := doRequest(ctx, m.addrs, path.Join(pdapi.Config, "placement-rule", name), "GET", nil) + if err == nil && res != nil { + err = json.Unmarshal(res, bundle) + } + return bundle, err +} + +// GetAllRuleBundles is used to get all rule bundles from PD. It is used to load full rules from PD while fullload infoschema. +func (m *PDPlacementManager) GetAllRuleBundles(ctx context.Context) ([]*placement.Bundle, error) { + var bundles []*placement.Bundle + res, err := doRequest(ctx, m.addrs, path.Join(pdapi.Config, "placement-rule"), "GET", nil) + if err == nil && res != nil { + err = json.Unmarshal(res, &bundles) + } + return bundles, err +} + +// PutRuleBundles is used to post specific rule bundles to PD. +func (m *PDPlacementManager) PutRuleBundles(ctx context.Context, bundles []*placement.Bundle) error { + if len(bundles) == 0 { + return nil + } + + b, err := json.Marshal(bundles) + if err != nil { + return err + } + + _, err = doRequest(ctx, m.addrs, path.Join(pdapi.Config, "placement-rule")+"?partial=true", "POST", bytes.NewReader(b)) + return err +} + +type mockPlacementManager struct { + sync.Mutex + bundles map[string]*placement.Bundle +} + +func (m *mockPlacementManager) GetRuleBundle(_ context.Context, name string) (*placement.Bundle, error) { + m.Lock() + defer m.Unlock() + + if bundle, ok := m.bundles[name]; ok { + return bundle, nil + } + + return &placement.Bundle{ID: name}, nil +} + +func (m *mockPlacementManager) GetAllRuleBundles(_ context.Context) ([]*placement.Bundle, error) { + m.Lock() + defer m.Unlock() + + bundles := make([]*placement.Bundle, 0, len(m.bundles)) + for _, bundle := range m.bundles { + bundles = append(bundles, bundle) + } + return bundles, nil +} + +func (m *mockPlacementManager) PutRuleBundles(_ context.Context, bundles []*placement.Bundle) error { + m.Lock() + defer m.Unlock() + + if m.bundles == nil { + m.bundles = make(map[string]*placement.Bundle) + } + + for _, bundle := range bundles { + if bundle.IsEmpty() { + delete(m.bundles, bundle.ID) + } else { + m.bundles[bundle.ID] = bundle + } + } + + return nil +} diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index a035889570f51..d89ba88a76ee0 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -168,23 +168,23 @@ func (do *Domain) checkEnableServerGlobalVar(name, sVal string) { var err error switch name { case variable.TiDBTSOClientBatchMaxWaitTime: - var val int64 - val, err = strconv.ParseInt(sVal, 10, 64) + var val float64 + val, err = strconv.ParseFloat(sVal, 64) if err != nil { break } - variable.MaxTSOBatchWaitInterval.Store(val) - err = do.SetPDClientDynamicOption(pd.MaxTSOBatchWaitInterval, time.Millisecond*time.Duration(val)) + err = do.SetPDClientDynamicOption(pd.MaxTSOBatchWaitInterval, time.Duration(float64(time.Millisecond)*val)) if err != nil { break } + variable.MaxTSOBatchWaitInterval.Store(val) case variable.TiDBEnableTSOFollowerProxy: val := variable.TiDBOptOn(sVal) - variable.EnableTSOFollowerProxy.Store(val) err = do.SetPDClientDynamicOption(pd.EnableTSOFollowerProxy, val) if err != nil { break } + variable.EnableTSOFollowerProxy.Store(val) case variable.TiDBEnableLocalTxn: variable.EnableLocalTxn.Store(variable.TiDBOptOn(sVal)) case variable.TiDBEnableStmtSummary: diff --git a/errors.toml b/errors.toml index e7e460f497fdf..f0a060045e7a8 100644 --- a/errors.toml +++ b/errors.toml @@ -1231,6 +1231,11 @@ error = ''' Operand should contain %d column(s) ''' +["planner:1242"] +error = ''' +Subquery returns more than 1 row +''' + ["planner:1247"] error = ''' Reference '%-.64s' not supported (%s) diff --git a/executor/admin_serial_test.go b/executor/admin_serial_test.go new file mode 100644 index 0000000000000..c9a60228ac7f8 --- /dev/null +++ b/executor/admin_serial_test.go @@ -0,0 +1,167 @@ +// 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 executor_test + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" +) + +func TestAdminCheckTableFailed(t *testing.T) { + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists admin_test") + tk.MustExec("create table admin_test (c1 int, c2 int, c3 varchar(255) default '1', primary key(c1), key(c3), unique key(c2), key(c2, c3))") + tk.MustExec("insert admin_test (c1, c2, c3) values (-10, -20, 'y'), (-1, -10, 'z'), (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") + + // Make some corrupted index. Build the index information. + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() + dbName := model.NewCIStr("test") + tblName := model.NewCIStr("admin_test") + tbl, err := is.TableByName(dbName, tblName) + require.NoError(t, err) + tblInfo := tbl.Meta() + idxInfo := tblInfo.Indices[1] + indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) + sc := ctx.GetSessionVars().StmtCtx + tk.Session().GetSessionVars().IndexLookupSize = 3 + tk.Session().GetSessionVars().MaxChunkSize = 3 + + // Reduce one row of index. + // Table count > index count. + // Index c2 is missing 11. + txn, err := store.Begin() + require.NoError(t, err) + err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), kv.IntHandle(-1)) + require.NoError(t, err) + err = txn.Commit(context.Background()) + 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)) + 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:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") + r := tk.MustQuery("admin recover index admin_test c2") + r.Check(testkit.Rows("1 7")) + tk.MustExec("admin check table admin_test") + + // Add one row of index. + // Table count < index count. + // Index c2 has one more values than table data: 0, and the handle 0 hasn't correlative record. + txn, err = store.Begin() + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(0), kv.IntHandle(0), nil) + require.NoError(t, err) + err = txn.Commit(context.Background()) + 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:") + 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:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") + + // Add one row of index. + // Table count < index count. + // Index c2 has two more values than table data: 10, 13, and these handles have correlative record. + txn, err = store.Begin() + require.NoError(t, err) + err = indexOpr.Delete(sc, txn, types.MakeDatums(0), kv.IntHandle(0)) + require.NoError(t, err) + // Make sure the index value "19" is smaller "21". Then we scan to "19" before "21". + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(19), kv.IntHandle(10), nil) + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(13), kv.IntHandle(2), nil) + require.NoError(t, err) + err = txn.Commit(context.Background()) + 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:") + 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:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") + + // Table count = index count. + // Two indices have the same handle. + txn, err = store.Begin() + require.NoError(t, err) + err = indexOpr.Delete(sc, txn, types.MakeDatums(13), kv.IntHandle(2)) + require.NoError(t, err) + err = indexOpr.Delete(sc, txn, types.MakeDatums(12), kv.IntHandle(2)) + require.NoError(t, err) + err = txn.Commit(context.Background()) + 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:") + 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:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") + + // Table count = index count. + // Index c2 has one line of data is 19, the corresponding table data is 20. + txn, err = store.Begin() + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(12), kv.IntHandle(2), nil) + require.NoError(t, err) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(10)) + require.NoError(t, err) + err = txn.Commit(context.Background()) + 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:") + 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:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") + + // Recover records. + txn, err = store.Begin() + require.NoError(t, err) + err = indexOpr.Delete(sc, txn, types.MakeDatums(19), kv.IntHandle(10)) + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(20), kv.IntHandle(10), nil) + require.NoError(t, err) + err = txn.Commit(context.Background()) + require.NoError(t, err) + tk.MustExec("admin check table admin_test") +} diff --git a/executor/admin_test.go b/executor/admin_test.go index 8707332688ad2..e984177ab4c52 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 PingCAP, Inc. +// 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. @@ -17,9 +17,9 @@ package executor_test import ( "context" "fmt" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" @@ -29,14 +29,19 @@ import ( "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/mock" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -func (s *testSuite1) TestAdminCheckIndexRange(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckIndexRange(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists check_index_test;`) tk.MustExec(`create table check_index_test (a int, b varchar(10), index a_b (a, b), index b (b))`) @@ -52,8 +57,13 @@ func (s *testSuite1) TestAdminCheckIndexRange(c *C) { result.Check(testkit.Rows("-1 hi 4", "2 cd 2")) } -func (s *testSuite5) TestAdminCheckIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckIndex(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") check := func() { tk.MustExec("insert admin_test (c1, c2) values (1, 1), (2, 2), (5, 5), (10, 10), (11, 11), (NULL, NULL)") @@ -78,16 +88,21 @@ func (s *testSuite5) TestAdminCheckIndex(c *C) { check() } -func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckIndexInTemporaryMode(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists temporary_admin_test;") tk.MustExec("create global temporary table temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2)) ON COMMIT DELETE ROWS;") tk.MustExec("insert temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") _, err := tk.Exec("admin check table temporary_admin_test;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin check table").Error()) + require.EqualError(t, err, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin check table").Error()) _, err = tk.Exec("admin check index temporary_admin_test c1;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin check index").Error()) + require.EqualError(t, err, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin check index").Error()) tk.MustExec("drop table if exists temporary_admin_test;") tk.MustExec("drop table if exists non_temporary_admin_test;") @@ -101,20 +116,25 @@ func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) { tk.MustExec("create global temporary table temporary_admin_checksum_table_with_index_test (id int, count int, PRIMARY KEY(id), KEY(count)) ON COMMIT DELETE ROWS;") tk.MustExec("create global temporary table temporary_admin_checksum_table_without_index_test (id int, count int, PRIMARY KEY(id)) ON COMMIT DELETE ROWS;") _, err = tk.Exec("admin checksum table temporary_admin_checksum_table_with_index_test;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) + require.EqualError(t, err, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) _, err = tk.Exec("admin checksum table temporary_admin_checksum_table_without_index_test;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) + require.EqualError(t, err, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) tk.MustExec("drop table if exists temporary_admin_checksum_table_with_index_test,temporary_admin_checksum_table_without_index_test;") } -func (s *testSuite5) TestAdminCheckIndexInLocalTemporaryMode(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckIndexInLocalTemporaryMode(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists local_temporary_admin_test;") tk.MustExec("create temporary table local_temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2))") tk.MustExec("insert local_temporary_admin_test (c1, c2) values (1,1), (2,2), (3,3);") _, err := tk.Exec("admin check table local_temporary_admin_test;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin check table").Error()) + require.EqualError(t, err, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin check table").Error()) tk.MustExec("drop table if exists temporary_admin_test;") tk.MustExec("drop table if exists local_temporary_admin_checksum_table_with_index_test;") @@ -122,14 +142,19 @@ func (s *testSuite5) TestAdminCheckIndexInLocalTemporaryMode(c *C) { tk.MustExec("create temporary table local_temporary_admin_checksum_table_with_index_test (id int, count int, PRIMARY KEY(id), KEY(count))") tk.MustExec("create temporary table local_temporary_admin_checksum_table_without_index_test (id int, count int, PRIMARY KEY(id))") _, err = tk.Exec("admin checksum table local_temporary_admin_checksum_table_with_index_test;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) + require.EqualError(t, err, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) _, err = tk.Exec("admin checksum table local_temporary_admin_checksum_table_without_index_test;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) + require.EqualError(t, err, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) tk.MustExec("drop table if exists local_temporary_admin_checksum_table_with_index_test,local_temporary_admin_checksum_table_without_index_test;") } -func (s *testSuite5) TestAdminCheckIndexInCacheTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckIndexInCacheTable(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists cache_admin_test;") tk.MustExec("create table cache_admin_test (c1 int, c2 int, c3 int default 1, index (c1), unique key(c2))") @@ -160,9 +185,13 @@ func (s *testSuite5) TestAdminCheckIndexInCacheTable(c *C) { tk.MustExec("admin checksum table cache_admin_table_without_index_test;") tk.MustExec("drop table if exists cache_admin_table_with_index_test,cache_admin_table_without_index_test;") } +func TestAdminRecoverIndex(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() -func (s *testSuite5) TestAdminRecoverIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") tk.MustExec("create table admin_test (c1 int, c2 int, c3 int default 1, index (c1), unique key(c2))") @@ -183,36 +212,36 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { // pk is handle, no additional unique index, no way to recover _, err := tk.Exec("admin recover index admin_test c1") // err:index is not found - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("admin recover index admin_test c2") r.Check(testkit.Rows("0 5")) tk.MustExec("admin check index admin_test c2") // Make some corrupted index. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("c2") indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - sc := s.ctx.GetSessionVars().StmtCtx - txn, err := s.store.Begin() - c.Assert(err, IsNil) + sc := ctx.GetSessionVars().StmtCtx + txn, err := store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, executor.ErrAdminCheckTable.Equal(err)) err = tk.ExecToErr("admin check index admin_test c2") - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") r.Check(testkit.Rows("4")) @@ -225,39 +254,39 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { tk.MustExec("admin check index admin_test c2") tk.MustExec("admin check table admin_test") - txn, err = s.store.Begin() - c.Assert(err, IsNil) + txn, err = store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check index admin_test c2") - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("admin recover index admin_test c2") r.Check(testkit.Rows("1 5")) tk.MustExec("admin check index admin_test c2") tk.MustExec("admin check table admin_test") - txn, err = s.store.Begin() - c.Assert(err, IsNil) + txn, err = store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) - c.Assert(err, IsNil) + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(2), kv.IntHandle(2)) - c.Assert(err, IsNil) + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(3), kv.IntHandle(3)) - c.Assert(err, IsNil) + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) - c.Assert(err, IsNil) + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(20)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test c2") - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") r.Check(testkit.Rows("0")) @@ -275,12 +304,17 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusteredIndexAdminRecoverIndex(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test_cluster_index_admin_recover;") tk.MustExec("create database test_cluster_index_admin_recover;") tk.MustExec("use test_cluster_index_admin_recover;") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn dbName := model.NewCIStr("test_cluster_index_admin_recover") tblName := model.NewCIStr("t") @@ -291,24 +325,24 @@ func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) { tk.MustQuery("admin recover index t `idx`;").Check(testkit.Rows("0 3")) tk.MustExec("admin check table t;") - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("idx") indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - sc := s.ctx.GetSessionVars().StmtCtx + sc := ctx.GetSessionVars().StmtCtx // Some index entries are missed. - txn, err := s.store.Begin() - c.Assert(err, IsNil) - cHandle := testutil.MustNewCommonHandle(c, "1", "3") + txn, err := store.Begin() + require.NoError(t, err) + cHandle := testkit.MustNewCommonHandle(t, "1", "3") err = indexOpr.Delete(sc, txn, types.MakeDatums(2), cHandle) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustGetErrCode("admin check table t", mysql.ErrAdminCheckTable) tk.MustGetErrCode("admin check index t idx", mysql.ErrAdminCheckTable) @@ -318,33 +352,39 @@ func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) { tk.MustExec("admin check table t;") } -func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminRecoverPartitionTableIndex(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") getTable := func() table.Table { - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) return tbl } checkFunc := func(tbl table.Table, pid int64, idxValue int) { idxInfo := tbl.Meta().FindIndexByName("c2") indexOpr := tables.NewIndex(pid, tbl.Meta(), idxInfo) - sc := s.ctx.GetSessionVars().StmtCtx - txn, err := s.store.Begin() - c.Assert(err, IsNil) + ctx := mock.NewContext() + sc := ctx.GetSessionVars().StmtCtx + txn, err := store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(idxValue), kv.IntHandle(idxValue)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, executor.ErrAdminCheckTable.Equal(err)) r := tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") r.Check(testkit.Rows("2")) @@ -365,7 +405,7 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { r.Check(testkit.Rows("0 3")) tbl := getTable() pi := tbl.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) + require.NotNil(t, pi) for i, p := range pi.Definitions { checkFunc(tbl, p.ID, i) } @@ -381,49 +421,54 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { r.Check(testkit.Rows("0 3")) tbl = getTable() pi = tbl.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) + require.NotNil(t, pi) for i, p := range pi.Definitions { checkFunc(tbl, p.ID, i*6) } } -func (s *testSuite5) TestAdminRecoverIndex1(c *C) { - tk := testkit.NewTestKit(c, s.store) - s.ctx = mock.NewContext() - s.ctx.Store = s.store +func TestAdminRecoverIndex1(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + ctx := mock.NewContext() + ctx.Store = store dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test") - sc := s.ctx.GetSessionVars().StmtCtx + sc := ctx.GetSessionVars().StmtCtx tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table admin_test (c1 varchar(255), c2 int, c3 int default 1, primary key(c1), unique key(c2))") tk.MustExec("insert admin_test (c1, c2) values ('1', 1), ('2', 2), ('3', 3), ('10', 10), ('20', 20)") r := tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(`primary`)") r.Check(testkit.Rows("5")) - is := s.domain.InfoSchema() + is := domain.InfoSchema() tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("primary") - c.Assert(idxInfo, NotNil) + require.NotNil(t, idxInfo) indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - txn, err := s.store.Begin() - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums("1"), kv.IntHandle(1)) - c.Assert(err, IsNil) + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums("2"), kv.IntHandle(2)) - c.Assert(err, IsNil) + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums("3"), kv.IntHandle(3)) - c.Assert(err, IsNil) + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums("10"), kv.IntHandle(4)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(`primary`)") r.Check(testkit.Rows("1")) @@ -439,8 +484,13 @@ func (s *testSuite5) TestAdminRecoverIndex1(c *C) { tk.MustExec("admin check index admin_test `primary`") } -func (s *testSuite5) TestAdminCleanupIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCleanupIndex(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") tk.MustExec("create table admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), unique key(c2), key (c3))") @@ -449,20 +499,20 @@ func (s *testSuite5) TestAdminCleanupIndex(c *C) { // pk is handle, no need to cleanup _, err := tk.Exec("admin cleanup index admin_test `primary`") - c.Assert(err, NotNil) + require.Error(t, err) r := tk.MustQuery("admin cleanup index admin_test c2") r.Check(testkit.Rows("0")) r = tk.MustQuery("admin cleanup index admin_test c3") r.Check(testkit.Rows("0")) // Make some dangling index. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo2 := tblInfo.FindIndexByName("c2") @@ -470,31 +520,31 @@ func (s *testSuite5) TestAdminCleanupIndex(c *C) { idxInfo3 := tblInfo.FindIndexByName("c3") indexOpr3 := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo3) - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(1), kv.IntHandle(-100), nil) - c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(6), kv.IntHandle(100), nil) - c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(8), kv.IntHandle(100), nil) - c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), kv.IntHandle(101), nil) - c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), kv.IntHandle(102), nil) - c.Assert(err, IsNil) - _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), kv.IntHandle(200), nil) - c.Assert(err, IsNil) - _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), kv.IntHandle(-200), nil) - c.Assert(err, IsNil) - _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(8), kv.IntHandle(-200), nil) - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) + _, err = indexOpr2.Create(ctx, txn, types.MakeDatums(1), kv.IntHandle(-100), nil) + require.NoError(t, err) + _, err = indexOpr2.Create(ctx, txn, types.MakeDatums(6), kv.IntHandle(100), nil) + require.NoError(t, err) + _, err = indexOpr2.Create(ctx, txn, types.MakeDatums(8), kv.IntHandle(100), nil) + require.NoError(t, err) + _, err = indexOpr2.Create(ctx, txn, types.MakeDatums(nil), kv.IntHandle(101), nil) + require.NoError(t, err) + _, err = indexOpr2.Create(ctx, txn, types.MakeDatums(nil), kv.IntHandle(102), nil) + require.NoError(t, err) + _, err = indexOpr3.Create(ctx, txn, types.MakeDatums(6), kv.IntHandle(200), nil) + require.NoError(t, err) + _, err = indexOpr3.Create(ctx, txn, types.MakeDatums(6), kv.IntHandle(-200), nil) + require.NoError(t, err) + _, err = indexOpr3.Create(ctx, txn, types.MakeDatums(8), kv.IntHandle(-200), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test c2") - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") r.Check(testkit.Rows("11")) r = tk.MustQuery("admin cleanup index admin_test c2") @@ -504,9 +554,9 @@ func (s *testSuite5) TestAdminCleanupIndex(c *C) { tk.MustExec("admin check index admin_test c2") err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test c3") - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c3)") r.Check(testkit.Rows("9")) r = tk.MustQuery("admin cleanup index admin_test c3") @@ -518,18 +568,23 @@ func (s *testSuite5) TestAdminCleanupIndex(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite5) TestAdminCleanupIndexForPartitionTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCleanupIndexForPartitionTable(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") getTable := func() table.Table { - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) return tbl } @@ -539,17 +594,18 @@ func (s *testSuite5) TestAdminCleanupIndexForPartitionTable(c *C) { idxInfo3 := tbl.Meta().FindIndexByName("c3") indexOpr3 := tables.NewIndex(pid, tbl.Meta(), idxInfo3) - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(idxValue), kv.IntHandle(handle), nil) - c.Assert(err, IsNil) - _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(idxValue), kv.IntHandle(handle), nil) - c.Assert(err, IsNil) + txn, err := store.Begin() + ctx := mock.NewContext() + require.NoError(t, err) + _, err = indexOpr2.Create(ctx, txn, types.MakeDatums(idxValue), kv.IntHandle(handle), nil) + require.NoError(t, err) + _, err = indexOpr3.Create(ctx, txn, types.MakeDatums(idxValue), kv.IntHandle(handle), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) r := tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") r.Check(testkit.Rows("4")) @@ -575,7 +631,7 @@ func (s *testSuite5) TestAdminCleanupIndexForPartitionTable(c *C) { r.Check(testkit.Rows("0")) tbl := getTable() pi := tbl.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) + require.NotNil(t, pi) for i, p := range pi.Definitions { checkFunc(tbl, p.ID, i+6, i+6) } @@ -591,17 +647,22 @@ func (s *testSuite5) TestAdminCleanupIndexForPartitionTable(c *C) { r.Check(testkit.Rows("0")) tbl = getTable() pi = tbl.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) + require.NotNil(t, pi) for i, p := range pi.Definitions { checkFunc(tbl, p.ID, i*6+1, i*6+1) } } -func (s *testSuite5) TestAdminCleanupIndexPKNotHandle(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCleanupIndexPKNotHandle(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table admin_test (c1 int, c2 int, c3 int, primary key (c1, c2))") tk.MustExec("insert admin_test (c1, c2) values (1, 2), (3, 4), (-5, 5)") @@ -609,33 +670,33 @@ func (s *testSuite5) TestAdminCleanupIndexPKNotHandle(c *C) { r.Check(testkit.Rows("0")) // Make some dangling index. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("primary") indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(7, 10), kv.IntHandle(-100), nil) - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(4, 6), kv.IntHandle(100), nil) - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(-7, 4), kv.IntHandle(101), nil) - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(7, 10), kv.IntHandle(-100), nil) + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(4, 6), kv.IntHandle(100), nil) + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(-7, 4), kv.IntHandle(101), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test `primary`") - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(`primary`)") r.Check(testkit.Rows("6")) r = tk.MustQuery("admin cleanup index admin_test `primary`") @@ -646,8 +707,13 @@ func (s *testSuite5) TestAdminCleanupIndexPKNotHandle(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite5) TestAdminCleanupIndexMore(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCleanupIndexMore(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") tk.MustExec("create table admin_test (c1 int, c2 int, unique key (c1, c2), key (c2))") @@ -657,13 +723,13 @@ func (s *testSuite5) TestAdminCleanupIndexMore(c *C) { tk.MustExec("admin cleanup index admin_test c2") // Make some dangling index. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo1 := tblInfo.FindIndexByName("c1") @@ -671,25 +737,25 @@ func (s *testSuite5) TestAdminCleanupIndexMore(c *C) { idxInfo2 := tblInfo.FindIndexByName("c2") indexOpr2 := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo2) - txn, err := s.store.Begin() - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) for i := 0; i < 2000; i++ { c1 := int64(2*i + 7) c2 := int64(2*i + 8) - _, err = indexOpr1.Create(s.ctx, txn, types.MakeDatums(c1, c2), kv.IntHandle(c1), nil) - c.Assert(err, IsNil, Commentf(errors.ErrorStack(err))) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(c2), kv.IntHandle(c1), nil) - c.Assert(err, IsNil) + _, err = indexOpr1.Create(ctx, txn, types.MakeDatums(c1, c2), kv.IntHandle(c1), nil) + require.NoErrorf(t, err, errors.ErrorStack(err)) + _, err = indexOpr2.Create(ctx, txn, types.MakeDatums(c2), kv.IntHandle(c1), nil) + require.NoError(t, err) } err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test c1") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test c2") - c.Assert(err, NotNil) + require.Error(t, err) r := tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX()") r.Check(testkit.Rows("3")) r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c1)") @@ -709,11 +775,16 @@ func (s *testSuite5) TestAdminCleanupIndexMore(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite5) TestClusteredAdminCleanupIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusteredAdminCleanupIndex(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table admin_test (c1 varchar(255), c2 int, c3 char(10) default 'c3', primary key (c1, c3), unique key(c2), key (c3))") tk.MustExec("insert admin_test (c1, c2) values ('c1_1', 2), ('c1_2', 4), ('c1_3', NULL)") tk.MustExec("insert admin_test (c1, c3) values ('c1_4', 'c3_4'), ('c1_5', 'c3_5'), ('c1_6', default)") @@ -724,10 +795,10 @@ func (s *testSuite5) TestClusteredAdminCleanupIndex(c *C) { tk.MustQuery("admin cleanup index admin_test `c3`").Check(testkit.Rows("0")) // Make some dangling index. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - tbl, err := s.domain.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("admin_test")) - c.Assert(err, IsNil) + ctx := mock.NewContext() + ctx.Store = store + tbl, err := domain.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("admin_test")) + require.NoError(t, err) // cleanup clustered primary key takes no effect. tblInfo := tbl.Meta() @@ -740,44 +811,44 @@ func (s *testSuite5) TestClusteredAdminCleanupIndex(c *C) { handle kv.Handle idxVal []types.Datum }{ - {testutil.MustNewCommonHandle(c, "c1_10", "c3_10"), types.MakeDatums(10)}, - {testutil.MustNewCommonHandle(c, "c1_10", "c3_11"), types.MakeDatums(11)}, - {testutil.MustNewCommonHandle(c, "c1_12", "c3_12"), types.MakeDatums(12)}, + {testkit.MustNewCommonHandle(t, "c1_10", "c3_10"), types.MakeDatums(10)}, + {testkit.MustNewCommonHandle(t, "c1_10", "c3_11"), types.MakeDatums(11)}, + {testkit.MustNewCommonHandle(t, "c1_12", "c3_12"), types.MakeDatums(12)}, } c3DanglingIdx := []struct { handle kv.Handle idxVal []types.Datum }{ - {testutil.MustNewCommonHandle(c, "c1_13", "c3_13"), types.MakeDatums("c3_13")}, - {testutil.MustNewCommonHandle(c, "c1_14", "c3_14"), types.MakeDatums("c3_14")}, - {testutil.MustNewCommonHandle(c, "c1_15", "c3_15"), types.MakeDatums("c3_15")}, + {testkit.MustNewCommonHandle(t, "c1_13", "c3_13"), types.MakeDatums("c3_13")}, + {testkit.MustNewCommonHandle(t, "c1_14", "c3_14"), types.MakeDatums("c3_14")}, + {testkit.MustNewCommonHandle(t, "c1_15", "c3_15"), types.MakeDatums("c3_15")}, } - txn, err := s.store.Begin() - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) for _, di := range c2DanglingIdx { - _, err := indexOpr2.Create(s.ctx, txn, di.idxVal, di.handle, nil) - c.Assert(err, IsNil) + _, err := indexOpr2.Create(ctx, txn, di.idxVal, di.handle, nil) + require.NoError(t, err) } for _, di := range c3DanglingIdx { - _, err := indexOpr3.Create(s.ctx, txn, di.idxVal, di.handle, nil) - c.Assert(err, IsNil) + _, err := indexOpr3.Create(ctx, txn, di.idxVal, di.handle, nil) + require.NoError(t, err) } err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test c2") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)").Check(testkit.Rows("9")) tk.MustQuery("admin cleanup index admin_test c2").Check(testkit.Rows("3")) tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)").Check(testkit.Rows("6")) tk.MustExec("admin check index admin_test c2") err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_test c3") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c3)").Check(testkit.Rows("9")) tk.MustQuery("admin cleanup index admin_test c3").Check(testkit.Rows("3")) tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c3)").Check(testkit.Rows("6")) @@ -785,8 +856,13 @@ func (s *testSuite5) TestClusteredAdminCleanupIndex(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckPartitionTableFailed(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test_p") tk.MustExec("create table admin_test_p (c1 int key,c2 int,c3 int,index idx(c2)) partition by hash(c1) partitions 4") @@ -794,45 +870,45 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { tk.MustExec("admin check table admin_test_p") // Make some corrupted index. Build the index information. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_test_p") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.Indices[0] - sc := s.ctx.GetSessionVars().StmtCtx - tk.Se.GetSessionVars().IndexLookupSize = 3 - tk.Se.GetSessionVars().MaxChunkSize = 3 + sc := ctx.GetSessionVars().StmtCtx + tk.Session().GetSessionVars().IndexLookupSize = 3 + tk.Session().GetSessionVars().MaxChunkSize = 3 // Reduce one row of index on partitions. // Table count > index count. for i := 0; i <= 5; i++ { partitionIdx := i % len(tblInfo.GetPartitionInfo().Definitions) indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) - txn, err := s.store.Begin() - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(i), kv.IntHandle(i)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test_p") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, 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)) - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + 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)) // TODO: fix admin recover for partition table. // r := tk.MustQuery("admin recover index admin_test_p idx") // r.Check(testkit.Rows("0 0")) // tk.MustExec("admin check table admin_test_p") // Manual recover index. - txn, err = s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i), kv.IntHandle(i), nil) - c.Assert(err, IsNil) + txn, err = store.Begin() + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(i), kv.IntHandle(i), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("admin check table admin_test_p") } @@ -841,22 +917,21 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { for i := 0; i <= 5; i++ { partitionIdx := i % len(tblInfo.GetPartitionInfo().Definitions) indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i+8), kv.IntHandle(i+8), nil) - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(i+8), kv.IntHandle(i+8), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test_p") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, 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.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)) // TODO: fix admin recover for partition table. - txn, err = s.store.Begin() - c.Assert(err, IsNil) + txn, err = store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i+8)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("admin check table admin_test_p") } @@ -864,165 +939,33 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { for i := 0; i <= 5; i++ { partitionIdx := i % len(tblInfo.GetPartitionInfo().Definitions) indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i+8), kv.IntHandle(i), nil) - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(i+8), kv.IntHandle(i), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test_p") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, 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.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)) // TODO: fix admin recover for partition table. - txn, err = s.store.Begin() - c.Assert(err, IsNil) + txn, err = store.Begin() + require.NoError(t, err) err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i)) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("admin check table admin_test_p") } } -func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists admin_test") - tk.MustExec("create table admin_test (c1 int, c2 int, c3 varchar(255) default '1', primary key(c1), key(c3), unique key(c2), key(c2, c3))") - tk.MustExec("insert admin_test (c1, c2, c3) values (-10, -20, 'y'), (-1, -10, 'z'), (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") - - // Make some corrupted index. Build the index information. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() - dbName := model.NewCIStr("test") - tblName := model.NewCIStr("admin_test") - tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) - tblInfo := tbl.Meta() - idxInfo := tblInfo.Indices[1] - indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - sc := s.ctx.GetSessionVars().StmtCtx - tk.Se.GetSessionVars().IndexLookupSize = 3 - tk.Se.GetSessionVars().MaxChunkSize = 3 - - // Reduce one row of index. - // Table count > index count. - // Index c2 is missing 11. - txn, err := s.store.Begin() - c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), kv.IntHandle(-1)) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, - "[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)}}}") - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) - tk.MustExec("set @@tidb_redact_log=1;") - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8003]admin_test err:[admin:8223]index:\"?\" != record:\"?\"") - tk.MustExec("set @@tidb_redact_log=0;") - r := tk.MustQuery("admin recover index admin_test c2") - r.Check(testkit.Rows("1 7")) - tk.MustExec("admin check table admin_test") - - // Add one row of index. - // Table count < index count. - // Index c2 has one more values than table data: 0, and the handle 0 hasn't correlative record. - txn, err = s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(0), kv.IntHandle(0), nil) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8133]handle 0, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:0, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") - tk.MustExec("set @@tidb_redact_log=1;") - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8133]handle \"?\", index:\"?\" != record:\"?\"") - tk.MustExec("set @@tidb_redact_log=0;") +func TestAdminCheckTable(t *testing.T) { + // test NULL value. + t.Parallel() - // Add one row of index. - // Table count < index count. - // Index c2 has two more values than table data: 10, 13, and these handles have correlative record. - txn, err = s.store.Begin() - c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(0), kv.IntHandle(0)) - c.Assert(err, IsNil) - // Make sure the index value "19" is smaller "21". Then we scan to "19" before "21". - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(19), kv.IntHandle(10), nil) - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(13), kv.IntHandle(2), nil) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[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:") - tk.MustExec("set @@tidb_redact_log=1;") - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") - tk.MustExec("set @@tidb_redact_log=0;") - - // Table count = index count. - // Two indices have the same handle. - txn, err = s.store.Begin() - c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(13), kv.IntHandle(2)) - c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(12), kv.IntHandle(2)) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[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:") - tk.MustExec("set @@tidb_redact_log=1;") - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") - tk.MustExec("set @@tidb_redact_log=0;") - - // Table count = index count. - // Index c2 has one line of data is 19, the corresponding table data is 20. - txn, err = s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), kv.IntHandle(2), nil) - c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(10)) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[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:") - tk.MustExec("set @@tidb_redact_log=1;") - err = tk.ExecToErr("admin check table admin_test") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") - tk.MustExec("set @@tidb_redact_log=0;") - - // Recover records. - txn, err = s.store.Begin() - c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(19), kv.IntHandle(10)) - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(20), kv.IntHandle(10), nil) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - tk.MustExec("admin check table admin_test") -} + store, clean := testkit.CreateMockStore(t) + defer clean() -func (s *testSuite8) TestAdminCheckTable(c *C) { - // test NULL value. - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`CREATE TABLE test_null ( a int(11) NOT NULL, @@ -1092,7 +1035,7 @@ func (s *testSuite8) TestAdminCheckTable(c *C) { tk.MustExec("use mysql") tk.MustExec(`admin check table test.t;`) err := tk.ExecToErr("admin check table t") - c.Assert(err, NotNil) + require.Error(t, err) // test add index on time type column which have default value tk.MustExec("use test") @@ -1132,21 +1075,31 @@ func (s *testSuite8) TestAdminCheckTable(c *C) { tk.MustExec(`create table t1 (a decimal(2,1), index(a))`) tk.MustExec(`insert into t1 set a='1.9'`) err = tk.ExecToErr(`alter table t1 modify column a decimal(3,2);`) - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec(`delete from t1;`) tk.MustExec(`admin check table t1;`) } -func (s *testSuite1) TestAdminCheckPrimaryIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckPrimaryIndex(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a bigint unsigned primary key, b int, c int, index idx(a, b));") tk.MustExec("insert into t values(1, 1, 1), (9223372036854775807, 2, 2);") tk.MustExec("admin check index t idx;") } -func (s *testSuite5) TestAdminCheckWithSnapshot(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdminCheckWithSnapshot(t *testing.T) { + t.Parallel() + + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_t_s") tk.MustExec("create table admin_t_s (a int, b int, key(a));") @@ -1156,27 +1109,27 @@ func (s *testSuite5) TestAdminCheckWithSnapshot(c *C) { snapshotTime := time.Now() - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() + ctx := mock.NewContext() + ctx.Store = store + is := domain.InfoSchema() dbName := model.NewCIStr("test") tblName := model.NewCIStr("admin_t_s") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("a") idxOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = idxOpr.Create(s.ctx, txn, types.MakeDatums(2), kv.IntHandle(100), nil) - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) + _, err = idxOpr.Create(ctx, txn, types.MakeDatums(2), kv.IntHandle(100), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) err = tk.ExecToErr("admin check table admin_t_s") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_t_s a") - c.Assert(err, NotNil) + require.Error(t, err) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" @@ -1193,9 +1146,9 @@ func (s *testSuite5) TestAdminCheckWithSnapshot(c *C) { tk.MustExec("set @@tidb_snapshot = ''") err = tk.ExecToErr("admin check table admin_t_s") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("admin check index admin_t_s a") - c.Assert(err, NotNil) + require.Error(t, err) r := tk.MustQuery("admin cleanup index admin_t_s a") r.Check(testkit.Rows("1")) diff --git a/executor/aggfuncs/aggfunc_test.go b/executor/aggfuncs/aggfunc_test.go index fb6e0ad05c041..f5ed94e821d63 100644 --- a/executor/aggfuncs/aggfunc_test.go +++ b/executor/aggfuncs/aggfunc_test.go @@ -466,109 +466,6 @@ func testMergePartialResult(t *testing.T, p aggTest) { require.Equalf(t, 0, result, "%v != %v", dt.String(), p.results[2]) } -// Deprecated: migrating to testMergePartialResult(t *testing.T, p aggTest) -func (s *testSuite) testMergePartialResult(c *C, p aggTest) { - srcChk := p.genSrcChk() - iter := chunk.NewIterator4Chunk(srcChk) - - args := []expression.Expression{&expression.Column{RetType: p.dataType, Index: 0}} - if p.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) - } - desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) - c.Assert(err, IsNil) - if p.orderBy { - desc.OrderByItems = []*util.ByItems{ - {Expr: args[0], Desc: true}, - } - } - partialDesc, finalDesc := desc.Split([]int{0, 1}) - - // build partial func for partial phase. - partialFunc := aggfuncs.Build(s.ctx, partialDesc, 0) - partialResult, _ := partialFunc.AllocPartialResult() - - // build final func for final phase. - finalFunc := aggfuncs.Build(s.ctx, finalDesc, 0) - finalPr, _ := finalFunc.AllocPartialResult() - resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{p.dataType}, 1) - if p.funcName == ast.AggFuncApproxCountDistinct { - resultChk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeString)}, 1) - } - if p.funcName == ast.AggFuncJsonArrayagg { - resultChk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeJSON)}, 1) - } - - // update partial result. - for row := iter.Begin(); row != iter.End(); row = iter.Next() { - _, err = partialFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialResult) - c.Assert(err, IsNil) - } - p.messUpChunk(srcChk) - err = partialFunc.AppendFinalResult2Chunk(s.ctx, partialResult, resultChk) - c.Assert(err, IsNil) - dt := resultChk.GetRow(0).GetDatum(0, p.dataType) - if p.funcName == ast.AggFuncApproxCountDistinct { - dt = resultChk.GetRow(0).GetDatum(0, types.NewFieldType(mysql.TypeString)) - } - if p.funcName == ast.AggFuncJsonArrayagg { - dt = resultChk.GetRow(0).GetDatum(0, types.NewFieldType(mysql.TypeJSON)) - } - result, err := dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) - c.Assert(err, IsNil) - c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) - - _, err = finalFunc.MergePartialResult(s.ctx, partialResult, finalPr) - c.Assert(err, IsNil) - partialFunc.ResetPartialResult(partialResult) - - srcChk = p.genSrcChk() - iter = chunk.NewIterator4Chunk(srcChk) - iter.Begin() - iter.Next() - for row := iter.Next(); row != iter.End(); row = iter.Next() { - _, err = partialFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialResult) - c.Assert(err, IsNil) - } - p.messUpChunk(srcChk) - resultChk.Reset() - err = partialFunc.AppendFinalResult2Chunk(s.ctx, partialResult, resultChk) - c.Assert(err, IsNil) - dt = resultChk.GetRow(0).GetDatum(0, p.dataType) - if p.funcName == ast.AggFuncApproxCountDistinct { - dt = resultChk.GetRow(0).GetDatum(0, types.NewFieldType(mysql.TypeString)) - } - if p.funcName == ast.AggFuncJsonArrayagg { - dt = resultChk.GetRow(0).GetDatum(0, types.NewFieldType(mysql.TypeJSON)) - } - result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) - c.Assert(err, IsNil) - c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) - _, err = finalFunc.MergePartialResult(s.ctx, partialResult, finalPr) - c.Assert(err, IsNil) - - if p.funcName == ast.AggFuncApproxCountDistinct { - resultChk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, 1) - } - if p.funcName == ast.AggFuncJsonArrayagg { - resultChk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeJSON)}, 1) - } - resultChk.Reset() - err = finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) - c.Assert(err, IsNil) - - dt = resultChk.GetRow(0).GetDatum(0, p.dataType) - if p.funcName == ast.AggFuncApproxCountDistinct { - dt = resultChk.GetRow(0).GetDatum(0, types.NewFieldType(mysql.TypeLonglong)) - } - if p.funcName == ast.AggFuncJsonArrayagg { - dt = resultChk.GetRow(0).GetDatum(0, types.NewFieldType(mysql.TypeJSON)) - } - result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[2]) - c.Assert(err, IsNil) - c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[2])) -} - func buildAggTester(funcName string, tp byte, numRows int, results ...interface{}) aggTest { return buildAggTesterWithFieldType(funcName, types.NewFieldType(tp), numRows, results...) } @@ -809,95 +706,6 @@ func testAggFunc(t *testing.T, p aggTest) { require.Equalf(t, 0, result, "%v != %v", dt.String(), p.results[0]) } -// Deprecated: migrating to func testAggFunc(t *testing.T, p aggTest) -func (s *testSuite) testAggFunc(c *C, p aggTest) { - srcChk := p.genSrcChk() - - args := []expression.Expression{&expression.Column{RetType: p.dataType, Index: 0}} - if p.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) - } - if p.funcName == ast.AggFuncApproxPercentile { - args = append(args, &expression.Constant{Value: types.NewIntDatum(50), RetType: types.NewFieldType(mysql.TypeLong)}) - } - desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) - c.Assert(err, IsNil) - if p.orderBy { - desc.OrderByItems = []*util.ByItems{ - {Expr: args[0], Desc: true}, - } - } - finalFunc := aggfuncs.Build(s.ctx, desc, 0) - finalPr, _ := finalFunc.AllocPartialResult() - resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) - - iter := chunk.NewIterator4Chunk(srcChk) - for row := iter.Begin(); row != iter.End(); row = iter.Next() { - _, err = finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) - c.Assert(err, IsNil) - } - p.messUpChunk(srcChk) - err = finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) - c.Assert(err, IsNil) - dt := resultChk.GetRow(0).GetDatum(0, desc.RetTp) - result, err := dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) - c.Assert(err, IsNil) - c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) - - // test the empty input - resultChk.Reset() - finalFunc.ResetPartialResult(finalPr) - err = finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) - c.Assert(err, IsNil) - dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) - result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) - c.Assert(err, IsNil) - c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) - - // test the agg func with distinct - desc, err = aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, true) - c.Assert(err, IsNil) - if p.orderBy { - desc.OrderByItems = []*util.ByItems{ - {Expr: args[0], Desc: true}, - } - } - finalFunc = aggfuncs.Build(s.ctx, desc, 0) - finalPr, _ = finalFunc.AllocPartialResult() - - resultChk.Reset() - srcChk = p.genSrcChk() - iter = chunk.NewIterator4Chunk(srcChk) - for row := iter.Begin(); row != iter.End(); row = iter.Next() { - _, err = finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) - c.Assert(err, IsNil) - } - p.messUpChunk(srcChk) - srcChk = p.genSrcChk() - iter = chunk.NewIterator4Chunk(srcChk) - for row := iter.Begin(); row != iter.End(); row = iter.Next() { - _, err = finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) - c.Assert(err, IsNil) - } - p.messUpChunk(srcChk) - err = finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) - c.Assert(err, IsNil) - dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) - result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) - c.Assert(err, IsNil) - c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) - - // test the empty input - resultChk.Reset() - finalFunc.ResetPartialResult(finalPr) - err = finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) - c.Assert(err, IsNil) - dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) - result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) - c.Assert(err, IsNil) - c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) -} - func testAggFuncWithoutDistinct(t *testing.T, p aggTest) { srcChk := p.genSrcChk() @@ -975,37 +783,96 @@ func testAggMemFunc(t *testing.T, p aggMemTest) { } } -// Deprecated: migrating to testAggMemFunc(t *testing.T, p aggMemTest) -func (s *testSuite) testAggMemFunc(c *C, p aggMemTest) { - srcChk := p.aggTest.genSrcChk() +func testMultiArgsAggFunc(t *testing.T, ctx sessionctx.Context, p multiArgsAggTest) { + srcChk := p.genSrcChk() - args := []expression.Expression{&expression.Column{RetType: p.aggTest.dataType, Index: 0}} - if p.aggTest.funcName == ast.AggFuncGroupConcat { + args := make([]expression.Expression, len(p.dataTypes)) + for k := 0; k < len(p.dataTypes); k++ { + args[k] = &expression.Column{RetType: p.dataTypes[k], Index: k} + } + if p.funcName == ast.AggFuncGroupConcat { args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } - desc, err := aggregation.NewAggFuncDesc(s.ctx, p.aggTest.funcName, args, p.isDistinct) - c.Assert(err, IsNil) - if p.aggTest.orderBy { + + desc, err := aggregation.NewAggFuncDesc(ctx, p.funcName, args, false) + require.NoError(t, err) + if p.orderBy { desc.OrderByItems = []*util.ByItems{ {Expr: args[0], Desc: true}, } } - finalFunc := aggfuncs.Build(s.ctx, desc, 0) - finalPr, memDelta := finalFunc.AllocPartialResult() - c.Assert(memDelta, Equals, p.allocMemDelta) + finalFunc := aggfuncs.Build(ctx, desc, 0) + finalPr, _ := finalFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) - updateMemDeltas, err := p.updateMemDeltaGens(srcChk, p.aggTest.dataType) - c.Assert(err, IsNil) iter := chunk.NewIterator4Chunk(srcChk) - i := 0 for row := iter.Begin(); row != iter.End(); row = iter.Next() { - memDelta, err := finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) - c.Assert(err, IsNil) - c.Assert(memDelta, Equals, updateMemDeltas[i]) - i++ + // FIXME: cannot assert error since there are cases of error, e.g. rows were cut by GROUPCONCAT + _, _ = finalFunc.UpdatePartialResult(ctx, []chunk.Row{row}, finalPr) + } + p.messUpChunk(srcChk) + err = finalFunc.AppendFinalResult2Chunk(ctx, finalPr, resultChk) + require.NoError(t, err) + dt := resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err := dt.CompareDatum(ctx.GetSessionVars().StmtCtx, &p.results[1]) + require.NoError(t, err) + require.Zerof(t, result, "%v != %v", dt.String(), p.results[1]) + + // test the empty input + resultChk.Reset() + finalFunc.ResetPartialResult(finalPr) + err = finalFunc.AppendFinalResult2Chunk(ctx, finalPr, resultChk) + require.NoError(t, err) + dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err = dt.CompareDatum(ctx.GetSessionVars().StmtCtx, &p.results[0]) + require.NoError(t, err) + require.Zerof(t, result, "%v != %v", dt.String(), p.results[0]) + + // test the agg func with distinct + desc, err = aggregation.NewAggFuncDesc(ctx, p.funcName, args, true) + require.NoError(t, err) + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc = aggfuncs.Build(ctx, desc, 0) + finalPr, _ = finalFunc.AllocPartialResult() + + resultChk.Reset() + srcChk = p.genSrcChk() + iter = chunk.NewIterator4Chunk(srcChk) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + // FIXME: cannot check error + _, _ = finalFunc.UpdatePartialResult(ctx, []chunk.Row{row}, finalPr) } + p.messUpChunk(srcChk) + srcChk = p.genSrcChk() + iter = chunk.NewIterator4Chunk(srcChk) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + // FIXME: cannot check error + _, _ = finalFunc.UpdatePartialResult(ctx, []chunk.Row{row}, finalPr) + } + p.messUpChunk(srcChk) + err = finalFunc.AppendFinalResult2Chunk(ctx, finalPr, resultChk) + require.NoError(t, err) + dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err = dt.CompareDatum(ctx.GetSessionVars().StmtCtx, &p.results[1]) + require.NoError(t, err) + require.Zerof(t, result, "%v != %v", dt.String(), p.results[1]) + + // test the empty input + resultChk.Reset() + finalFunc.ResetPartialResult(finalPr) + err = finalFunc.AppendFinalResult2Chunk(ctx, finalPr, resultChk) + require.NoError(t, err) + dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err = dt.CompareDatum(ctx.GetSessionVars().StmtCtx, &p.results[0]) + require.NoError(t, err) + require.Zero(t, result) } +// Deprecated: migrating to testMultiArgsAggFunc(t *testing.T, ctx sessionctx.Context, p multiArgsAggTest) func (s *testSuite) testMultiArgsAggFunc(c *C, p multiArgsAggTest) { srcChk := p.genSrcChk() @@ -1095,6 +962,41 @@ func (s *testSuite) testMultiArgsAggFunc(c *C, p multiArgsAggTest) { c.Assert(result, Equals, 0) } +func testMultiArgsAggMemFunc(t *testing.T, p multiArgsAggMemTest) { + srcChk := p.multiArgsAggTest.genSrcChk() + ctx := mock.NewContext() + + args := make([]expression.Expression, len(p.multiArgsAggTest.dataTypes)) + for k := 0; k < len(p.multiArgsAggTest.dataTypes); k++ { + args[k] = &expression.Column{RetType: p.multiArgsAggTest.dataTypes[k], Index: k} + } + if p.multiArgsAggTest.funcName == ast.AggFuncGroupConcat { + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) + } + + desc, err := aggregation.NewAggFuncDesc(ctx, p.multiArgsAggTest.funcName, args, p.isDistinct) + require.NoError(t, err) + if p.multiArgsAggTest.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc := aggfuncs.Build(ctx, desc, 0) + finalPr, memDelta := finalFunc.AllocPartialResult() + require.Equal(t, p.allocMemDelta, memDelta) + + updateMemDeltas, err := p.multiArgsUpdateMemDeltaGens(srcChk, p.multiArgsAggTest.dataTypes, desc.OrderByItems) + require.NoError(t, err) + iter := chunk.NewIterator4Chunk(srcChk) + i := 0 + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + memDelta, _ := finalFunc.UpdatePartialResult(ctx, []chunk.Row{row}, finalPr) + require.Equal(t, updateMemDeltas[i], memDelta) + i++ + } +} + +// Deprecated: migrating to testMultiArgsAggMemFunc(t *testing.T, p multiArgsAggMemTest) func (s *testSuite) testMultiArgsAggMemFunc(c *C, p multiArgsAggMemTest) { srcChk := p.multiArgsAggTest.genSrcChk() @@ -1127,7 +1029,6 @@ func (s *testSuite) testMultiArgsAggMemFunc(c *C, p multiArgsAggMemTest) { i++ } } - func (s *testSuite) benchmarkAggFunc(b *testing.B, p aggTest) { srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{p.dataType}, p.numRows) for i := 0; i < p.numRows; i++ { diff --git a/executor/aggfuncs/func_count_test.go b/executor/aggfuncs/func_count_test.go index a1a04c7438d79..603c46e4fba3b 100644 --- a/executor/aggfuncs/func_count_test.go +++ b/executor/aggfuncs/func_count_test.go @@ -16,16 +16,18 @@ package aggfuncs_test import ( "encoding/binary" + "fmt" "testing" "github.com/dgryski/go-farm" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/set" + "github.com/stretchr/testify/require" ) func genApproxDistinctMergePartialResult(begin, end uint64) string { @@ -39,15 +41,19 @@ func genApproxDistinctMergePartialResult(begin, end uint64) string { return string(o.Serialize()) } -func (s *testSuite) TestMergePartialResult4Count(c *C) { +func TestMergePartialResult4Count(t *testing.T) { + t.Parallel() + tester := buildAggTester(ast.AggFuncCount, mysql.TypeLonglong, 5, 5, 3, 8) - s.testMergePartialResult(c, tester) + testMergePartialResult(t, tester) tester = buildAggTester(ast.AggFuncApproxCountDistinct, mysql.TypeLonglong, 5, genApproxDistinctMergePartialResult(0, 5), genApproxDistinctMergePartialResult(2, 5), 5) - s.testMergePartialResult(c, tester) + testMergePartialResult(t, tester) } -func (s *testSuite) TestCount(c *C) { +func TestCount(t *testing.T) { + t.Parallel() + tests := []aggTest{ buildAggTester(ast.AggFuncCount, mysql.TypeLonglong, 5, 0, 5), buildAggTester(ast.AggFuncCount, mysql.TypeFloat, 5, 0, 5), @@ -58,8 +64,10 @@ func (s *testSuite) TestCount(c *C) { buildAggTester(ast.AggFuncCount, mysql.TypeDuration, 5, 0, 5), buildAggTester(ast.AggFuncCount, mysql.TypeJSON, 5, 0, 5), } - for _, test := range tests { - s.testAggFunc(c, test) + for i, test := range tests { + t.Run(fmt.Sprintf("%s_%d", test.funcName, i), func(t *testing.T) { + testAggFunc(t, test) + }) } tests2 := []multiArgsAggTest{ buildMultiArgsAggTester(ast.AggFuncCount, []byte{mysql.TypeLonglong, mysql.TypeLonglong}, mysql.TypeLonglong, 5, 0, 5), @@ -71,8 +79,10 @@ func (s *testSuite) TestCount(c *C) { buildMultiArgsAggTester(ast.AggFuncCount, []byte{mysql.TypeDuration, mysql.TypeDuration}, mysql.TypeLonglong, 5, 0, 5), buildMultiArgsAggTester(ast.AggFuncCount, []byte{mysql.TypeJSON, mysql.TypeJSON}, mysql.TypeLonglong, 5, 0, 5), } - for _, test := range tests2 { - s.testMultiArgsAggFunc(c, test) + for i, test := range tests2 { + t.Run(fmt.Sprintf("%s_%d", test.funcName, i), func(t *testing.T) { + testMultiArgsAggFunc(t, mock.NewContext(), test) + }) } tests3 := []aggTest{ @@ -86,7 +96,7 @@ func (s *testSuite) TestCount(c *C) { buildAggTester(ast.AggFuncCount, mysql.TypeJSON, 5, 0, 5), } for _, test := range tests3 { - s.testAggFunc(c, test) + testAggFunc(t, test) } tests4 := []multiArgsAggTest{ @@ -100,12 +110,16 @@ func (s *testSuite) TestCount(c *C) { buildMultiArgsAggTester(ast.AggFuncApproxCountDistinct, []byte{mysql.TypeJSON, mysql.TypeJSON}, mysql.TypeLonglong, 5, 0, 5), } - for _, test := range tests4 { - s.testMultiArgsAggFunc(c, test) + for i, test := range tests4 { + t.Run(fmt.Sprintf("%s_%d", test.funcName, i), func(t *testing.T) { + testMultiArgsAggFunc(t, mock.NewContext(), test) + }) } } -func (s *testSuite) TestMemCount(c *C) { +func TestMemCount(t *testing.T) { + t.Parallel() + tests := []aggMemTest{ buildAggMemTester(ast.AggFuncCount, mysql.TypeLonglong, 5, aggfuncs.DefPartialResult4CountSize, defaultUpdateMemDeltaGens, false), @@ -142,22 +156,26 @@ func (s *testSuite) TestMemCount(c *C) { buildAggMemTester(ast.AggFuncApproxCountDistinct, mysql.TypeString, 5, aggfuncs.DefPartialResult4ApproxCountDistinctSize, approxCountDistinctUpdateMemDeltaGens, true), } - for _, test := range tests { - s.testAggMemFunc(c, test) + for i, test := range tests { + t.Run(fmt.Sprintf("%s_%d", test.aggTest.funcName, i), func(t *testing.T) { + testAggMemFunc(t, test) + }) } } -func (s *testSuite) TestWriteTime(c *C) { - t, err := types.ParseDate(&(stmtctx.StatementContext{}), "2020-11-11") - c.Assert(err, IsNil) +func TestWriteTime(t *testing.T) { + t.Parallel() + + tt, err := types.ParseDate(&(stmtctx.StatementContext{}), "2020-11-11") + require.NoError(t, err) buf := make([]byte, 16) for i := range buf { buf[i] = uint8(255) } - aggfuncs.WriteTime(buf, t) + aggfuncs.WriteTime(buf, tt) for i := range buf { - c.Assert(buf[i] == uint8(255), IsFalse) + require.False(t, buf[i] == uint8(255)) } } diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index 1718b22682cfd..a7fb3b7359ca5 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -21,15 +21,14 @@ import ( "sync/atomic" "unsafe" - mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/expression" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/set" ) @@ -479,7 +478,7 @@ func (e *groupConcatOrder) UpdatePartialResult(sctx sessionctx.Context, rowsInGr func (e *groupConcatOrder) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) (memDelta int64, err error) { // If order by exists, the parallel hash aggregation is forbidden in executorBuilder.buildHashAgg. // So MergePartialResult will not be called. - return 0, dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("groupConcatOrder.MergePartialResult should not be called") + return 0, plannercore.ErrInternal.GenWithStack("groupConcatOrder.MergePartialResult should not be called") } // SetTruncated will be called in `executorBuilder#buildHashAgg` with duck-type. @@ -599,7 +598,7 @@ func (e *groupConcatDistinctOrder) UpdatePartialResult(sctx sessionctx.Context, func (e *groupConcatDistinctOrder) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) (memDelta int64, err error) { // If order by exists, the parallel hash aggregation is forbidden in executorBuilder.buildHashAgg. // So MergePartialResult will not be called. - return 0, dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("groupConcatDistinctOrder.MergePartialResult should not be called") + return 0, plannercore.ErrInternal.GenWithStack("groupConcatDistinctOrder.MergePartialResult should not be called") } // GetDatumMemSize calculates the memory size of each types.Datum in sortRow.byItems. diff --git a/executor/aggfuncs/func_group_concat_test.go b/executor/aggfuncs/func_group_concat_test.go index be51720e3db0d..f609de53f858e 100644 --- a/executor/aggfuncs/func_group_concat_test.go +++ b/executor/aggfuncs/func_group_concat_test.go @@ -17,8 +17,8 @@ package aggfuncs_test import ( "bytes" "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" @@ -28,37 +28,47 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/set" + "github.com/stretchr/testify/require" ) -func (s *testSuite) TestMergePartialResult4GroupConcat(c *C) { +func TestMergePartialResult4GroupConcat(t *testing.T) { + t.Parallel() + test := buildAggTester(ast.AggFuncGroupConcat, mysql.TypeString, 5, "0 1 2 3 4", "2 3 4", "0 1 2 3 4 2 3 4") - s.testMergePartialResult(c, test) + testMergePartialResult(t, test) } -func (s *testSuite) TestGroupConcat(c *C) { +func TestGroupConcat(t *testing.T) { + t.Parallel() + + ctx := mock.NewContext() + test := buildAggTester(ast.AggFuncGroupConcat, mysql.TypeString, 5, nil, "0 1 2 3 4") - s.testAggFunc(c, test) + testAggFunc(t, test) test2 := buildMultiArgsAggTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, nil, "44 33 22 11 00") test2.orderBy = true - s.testMultiArgsAggFunc(c, test2) + testMultiArgsAggFunc(t, ctx, test2) defer func() { - err := variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, "1024") - c.Assert(err, IsNil) + err := variable.SetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen, "1024") + require.NoError(t, err) }() // minimum GroupConcatMaxLen is 4 for i := 4; i <= 7; i++ { - err := variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, fmt.Sprint(i)) - c.Assert(err, IsNil) + err := variable.SetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen, fmt.Sprint(i)) + require.NoError(t, err) test2 = buildMultiArgsAggTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, nil, "44 33 22 11 00"[:i]) test2.orderBy = true - s.testMultiArgsAggFunc(c, test2) + testMultiArgsAggFunc(t, ctx, test2) } } -func (s *testSuite) TestMemGroupConcat(c *C) { +func TestMemGroupConcat(t *testing.T) { + t.Parallel() + multiArgsTest1 := buildMultiArgsAggMemTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, aggfuncs.DefPartialResult4GroupConcatSize+aggfuncs.DefBytesBufferSize, groupConcatMultiArgsUpdateMemDeltaGens, false) multiArgsTest2 := buildMultiArgsAggMemTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, @@ -72,8 +82,10 @@ func (s *testSuite) TestMemGroupConcat(c *C) { multiArgsTest4.multiArgsAggTest.orderBy = true multiArgsTests := []multiArgsAggMemTest{multiArgsTest1, multiArgsTest2, multiArgsTest3, multiArgsTest4} - for _, test := range multiArgsTests { - s.testMultiArgsAggMemFunc(c, test) + for i, test := range multiArgsTests { + t.Run(fmt.Sprintf("%s_%d", test.multiArgsAggTest.funcName, i), func(t *testing.T) { + testMultiArgsAggMemFunc(t, test) + }) } } diff --git a/executor/aggfuncs/func_stddevpop_test.go b/executor/aggfuncs/func_stddevpop_test.go index c3f2d688e9a0f..3dc95820351cd 100644 --- a/executor/aggfuncs/func_stddevpop_test.go +++ b/executor/aggfuncs/func_stddevpop_test.go @@ -15,25 +15,30 @@ package aggfuncs_test import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" ) -func (s *testSuite) TestMergePartialResult4Stddevpop(c *C) { +func TestMergePartialResult4Stddevpop(t *testing.T) { + t.Parallel() + tests := []aggTest{ buildAggTester(ast.AggFuncStddevPop, mysql.TypeDouble, 5, 1.4142135623730951, 0.816496580927726, 1.3169567191065923), } for _, test := range tests { - s.testMergePartialResult(c, test) + testMergePartialResult(t, test) } } -func (s *testSuite) TestStddevpop(c *C) { +func TestStddevpop(t *testing.T) { + t.Parallel() + tests := []aggTest{ buildAggTester(ast.AggFuncStddevPop, mysql.TypeDouble, 5, nil, 1.4142135623730951), } for _, test := range tests { - s.testAggFunc(c, test) + testAggFunc(t, test) } } diff --git a/executor/aggfuncs/func_stddevsamp_test.go b/executor/aggfuncs/func_stddevsamp_test.go index e8a45f688ab64..dc0f972c30712 100644 --- a/executor/aggfuncs/func_stddevsamp_test.go +++ b/executor/aggfuncs/func_stddevsamp_test.go @@ -15,25 +15,30 @@ package aggfuncs_test import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" ) -func (s *testSuite) TestMergePartialResult4Stddevsamp(c *C) { +func TestMergePartialResult4Stddevsamp(t *testing.T) { + t.Parallel() + tests := []aggTest{ buildAggTester(ast.AggFuncStddevSamp, mysql.TypeDouble, 5, 1.5811388300841898, 1, 1.407885953173359), } for _, test := range tests { - s.testMergePartialResult(c, test) + testMergePartialResult(t, test) } } -func (s *testSuite) TestStddevsamp(c *C) { +func TestStddevsamp(t *testing.T) { + t.Parallel() + tests := []aggTest{ buildAggTester(ast.AggFuncStddevSamp, mysql.TypeDouble, 5, nil, 1.5811388300841898), } for _, test := range tests { - s.testAggFunc(c, test) + testAggFunc(t, test) } } diff --git a/executor/aggfuncs/func_varsamp_test.go b/executor/aggfuncs/func_varsamp_test.go index c2cba624336e2..fceb8a2a1a559 100644 --- a/executor/aggfuncs/func_varsamp_test.go +++ b/executor/aggfuncs/func_varsamp_test.go @@ -15,25 +15,30 @@ package aggfuncs_test import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" ) -func (s *testSuite) TestMergePartialResult4Varsamp(c *C) { +func TestMergePartialResult4Varsamp(t *testing.T) { + t.Parallel() + tests := []aggTest{ buildAggTester(ast.AggFuncVarSamp, mysql.TypeDouble, 5, 2.5, 1, 1.9821428571428572), } for _, test := range tests { - s.testMergePartialResult(c, test) + testMergePartialResult(t, test) } } -func (s *testSuite) TestVarsamp(c *C) { +func TestVarsamp(t *testing.T) { + t.Parallel() + tests := []aggTest{ buildAggTester(ast.AggFuncVarSamp, mysql.TypeDouble, 5, nil, 2.5), } for _, test := range tests { - s.testAggFunc(c, test) + testAggFunc(t, test) } } diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 01efdf1097349..93c037b6775b6 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -78,8 +78,9 @@ type BatchPointGetExec struct { // virtualColumnRetFieldTypes records the RetFieldTypes of virtual columns. virtualColumnRetFieldTypes []*types.FieldType - snapshot kv.Snapshot - stats *runtimeStatsWithSnapshot + snapshot kv.Snapshot + stats *runtimeStatsWithSnapshot + cacheTable kv.MemBuffer } // buildVirtualColumnInfo saves virtual column indices and sort them in definition order @@ -115,6 +116,9 @@ func (e *BatchPointGetExec) Open(context.Context) error { } else { snapshot = e.ctx.GetSnapshotWithTS(e.snapshotTS) } + if e.cacheTable != nil { + snapshot = cacheTableSnapshot{snapshot, e.cacheTable} + } if e.runtimeStats != nil { snapshotStats := &txnsnapshot.SnapshotRuntimeStats{} e.stats = &runtimeStatsWithSnapshot{ @@ -162,6 +166,48 @@ func (e *BatchPointGetExec) Open(context.Context) error { return nil } +// CacheTable always use memBuffer in session as snapshot. +// cacheTableSnapshot inherits kv.Snapshot and override the BatchGet methods and Get methods. +type cacheTableSnapshot struct { + kv.Snapshot + memBuffer kv.MemBuffer +} + +func (s cacheTableSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]byte, error) { + values := make(map[string][]byte) + if s.memBuffer == nil { + return values, nil + } + + for _, key := range keys { + val, err := s.memBuffer.Get(ctx, key) + if kv.ErrNotExist.Equal(err) { + continue + } + + if err != nil { + return nil, err + } + + if len(val) == 0 { + continue + } + + values[string(key)] = val + } + + return values, nil +} + +func (s cacheTableSnapshot) Get(ctx context.Context, key kv.Key) ([]byte, error) { + return s.memBuffer.Get(ctx, key) +} + +// MockNewCacheTableSnapShot only serves for test. +func MockNewCacheTableSnapShot(snapshot kv.Snapshot, memBuffer kv.MemBuffer) *cacheTableSnapshot { + return &cacheTableSnapshot{snapshot, memBuffer} +} + // Close implements the Executor interface. func (e *BatchPointGetExec) Close() error { if e.runtimeStats != nil && e.snapshot != nil { diff --git a/executor/batch_point_get_test.go b/executor/batch_point_get_test.go index 5156a6a6392fe..b4a167e910cb0 100644 --- a/executor/batch_point_get_test.go +++ b/executor/batch_point_get_test.go @@ -15,14 +15,18 @@ package executor_test import ( + "context" "fmt" "sync" "testing" "time" + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/tikv" ) func TestBatchPointGetExec(t *testing.T) { @@ -337,3 +341,30 @@ func TestBatchPointGetIssue25167(t *testing.T) { tk.MustExec("insert into t values (1)") tk.MustQuery("select * from t as of timestamp @a where a in (1,2,3)").Check(testkit.Rows()) } + +func TestCacheSnapShot(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + se := tk.Session() + ctx := context.Background() + txn, err := se.GetStore().Begin(tikv.WithStartTS(0)) + memBuffer := txn.GetMemBuffer() + require.NoError(t, err) + var keys []kv.Key + for i := 0; i < 2; i++ { + keys = append(keys, []byte(string(rune(i)))) + } + err = memBuffer.Set(keys[0], []byte("1111")) + require.NoError(t, err) + err = memBuffer.Set(keys[1], []byte("2222")) + require.NoError(t, err) + cacheTableSnapShot := executor.MockNewCacheTableSnapShot(nil, memBuffer) + get, err := cacheTableSnapShot.Get(ctx, keys[0]) + require.NoError(t, err) + require.Equal(t, get, []byte("1111")) + batchGet, err := cacheTableSnapShot.BatchGet(ctx, keys) + require.NoError(t, err) + require.Equal(t, batchGet[string(keys[0])], []byte("1111")) + require.Equal(t, batchGet[string(keys[1])], []byte("2222")) +} diff --git a/executor/builder.go b/executor/builder.go index e10a2b6b11970..d42d584bbb982 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/diagnosticspb" + "github.com/pingcap/log" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor/aggfuncs" @@ -54,7 +55,6 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/cteutil" - "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" @@ -1041,6 +1041,7 @@ func (b *executorBuilder) buildUnionScanFromReader(reader Executor, v *plannerco return x } us := &UnionScanExec{baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), reader)} + us.cacheTable = v.CacheTable // Get the handle column index of the below Plan. us.belowHandleCols = v.HandleCols us.mutableRow = chunk.MutRowFromTypes(retTypes(us)) @@ -1264,7 +1265,7 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo // consider collations for i := range v.EqualConditions { - chs, coll := v.EqualConditions[i].CharsetAndCollation(e.ctx) + chs, coll := v.EqualConditions[i].CharsetAndCollation() leftTypes[i].Charset, leftTypes[i].Collate = chs, coll rightTypes[i].Charset, rightTypes[i].Collate = chs, coll } @@ -3135,8 +3136,7 @@ func prunePartitionForInnerExecutor(ctx sessionctx.Context, tbl table.Table, sch return nil, false, nil, nil } if lookUpContent[0].keyColIDs == nil { - return nil, false, nil, - dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("cannot get column IDs when dynamic pruning") + return nil, false, nil, plannercore.ErrInternal.GenWithStack("cannot get column IDs when dynamic pruning") } keyColOffsets := make([]int, len(lookUpContent[0].keyColIDs)) for i, colID := range lookUpContent[0].keyColIDs { @@ -3148,8 +3148,7 @@ func prunePartitionForInnerExecutor(ctx sessionctx.Context, tbl table.Table, sch } } if offset == -1 { - return nil, false, nil, - dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("invalid column offset when dynamic pruning") + return nil, false, nil, plannercore.ErrInternal.GenWithStack("invalid column offset when dynamic pruning") } keyColOffsets[i] = offset } @@ -4364,7 +4363,9 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan partTblID: plan.PartTblID, columns: plan.Columns, } - + if plan.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { + e.cacheTable = b.getCacheTable(plan.TblInfo, startTS) + } if plan.TblInfo.TempTableType != model.TempTableNone { // Temporary table should not do any lock operations e.lock = false @@ -4663,3 +4664,32 @@ func (b *executorBuilder) validCanReadTemporaryTable(tbl *model.TableInfo) error return nil } + +func (b *executorBuilder) getCacheTable(tblInfo *model.TableInfo, startTS uint64) kv.MemBuffer { + tbl, ok := b.is.TableByID(tblInfo.ID) + if !ok { + b.err = errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(b.ctx.GetSessionVars().CurrentDB, tblInfo.Name)) + return nil + } + cacheData := tbl.(table.CachedTable).TryReadFromCache(startTS) + if cacheData != nil { + b.ctx.GetSessionVars().StmtCtx.ReadFromTableCache = true + return cacheData + } + go func() { + defer func() { + if r := recover(); r != nil { + logutil.BgLogger().Error("panic in the recoverable goroutine", + zap.Reflect("r", r), + zap.Stack("stack trace")) + } + }() + if !b.ctx.GetSessionVars().StmtCtx.InExplainStmt { + err := tbl.(table.CachedTable).UpdateLockForRead(b.ctx.GetStore(), startTS) + if err != nil { + log.Warn("Update Lock Info Error") + } + } + }() + return nil +} diff --git a/executor/collation_test.go b/executor/collation_serial_test.go similarity index 75% rename from executor/collation_test.go rename to executor/collation_serial_test.go index d439a8df4fe02..83d0b7251d6bd 100644 --- a/executor/collation_test.go +++ b/executor/collation_serial_test.go @@ -1,4 +1,4 @@ -// Copyright 2020 PingCAP, Inc. +// 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. @@ -15,21 +15,18 @@ package executor import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/expression" "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" ) -var _ = SerialSuites(&testCollationSuite{}) - -type testCollationSuite struct { -} - -func (s *testCollationSuite) TestVecGroupChecker(c *C) { +func TestVecGroupChecker(t *testing.T) { collate.SetNewCollationEnabledForTest(true) defer collate.SetNewCollationEnabledForTest(false) @@ -53,35 +50,35 @@ func (s *testCollationSuite) TestVecGroupChecker(c *C) { tp.Collate = "bin" groupChecker.reset() _, err := groupChecker.splitIntoGroups(chk) - c.Assert(err, IsNil) + require.NoError(t, err) for i := 0; i < 6; i++ { b, e := groupChecker.getNextGroup() - c.Assert(b, Equals, i) - c.Assert(e, Equals, i+1) + require.Equal(t, b, i) + require.Equal(t, e, i+1) } - c.Assert(groupChecker.isExhausted(), IsTrue) + require.True(t, groupChecker.isExhausted()) tp.Collate = "utf8_general_ci" groupChecker.reset() _, err = groupChecker.splitIntoGroups(chk) - c.Assert(err, IsNil) + require.NoError(t, err) for i := 0; i < 3; i++ { b, e := groupChecker.getNextGroup() - c.Assert(b, Equals, i*2) - c.Assert(e, Equals, i*2+2) + require.Equal(t, b, i*2) + require.Equal(t, e, i*2+2) } - c.Assert(groupChecker.isExhausted(), IsTrue) + require.True(t, groupChecker.isExhausted()) tp.Collate = "utf8_unicode_ci" groupChecker.reset() _, err = groupChecker.splitIntoGroups(chk) - c.Assert(err, IsNil) + require.NoError(t, err) for i := 0; i < 3; i++ { b, e := groupChecker.getNextGroup() - c.Assert(b, Equals, i*2) - c.Assert(e, Equals, i*2+2) + require.Equal(t, b, i*2) + require.Equal(t, e, i*2+2) } - c.Assert(groupChecker.isExhausted(), IsTrue) + require.True(t, groupChecker.isExhausted()) // test padding tp.Collate = "utf8_bin" @@ -92,9 +89,9 @@ func (s *testCollationSuite) TestVecGroupChecker(c *C) { chk.Column(0).AppendString("a ") groupChecker.reset() _, err = groupChecker.splitIntoGroups(chk) - c.Assert(err, IsNil) + require.NoError(t, err) b, e := groupChecker.getNextGroup() - c.Assert(b, Equals, 0) - c.Assert(e, Equals, 3) - c.Assert(groupChecker.isExhausted(), IsTrue) + require.Equal(t, b, 0) + require.Equal(t, e, 3) + require.True(t, groupChecker.isExhausted()) } diff --git a/executor/concurrent_map_test.go b/executor/concurrent_map_test.go index ebf2da277ed4f..aa6a318874590 100644 --- a/executor/concurrent_map_test.go +++ b/executor/concurrent_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2020 PingCAP, Inc. +// 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. @@ -16,19 +16,21 @@ package executor import ( "sync" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/util/chunk" + "github.com/stretchr/testify/require" ) // TestConcurrentMap first inserts 1000 entries, then checks them -func (cm *pkgTestSuite) TestConcurrentMap(c *C) { +func TestConcurrentMap(t *testing.T) { + t.Parallel() m := newConcurrentMap() const iterations = 1000 const mod = 111 wg := &sync.WaitGroup{} wg.Add(2) - // Using go routines insert 1000 entires into the map. + // Using go routines insert 1000 entries into the map. go func() { defer wg.Done() for i := 0; i < iterations/2; i++ { @@ -50,17 +52,17 @@ func (cm *pkgTestSuite) TestConcurrentMap(c *C) { for i := 0; i < iterations; i++ { found := false for en, ok := m.Get(uint64(i % mod)); en != nil; en = en.next { - c.Assert(ok, IsTrue) + require.True(t, ok) if en.ptr.RowIdx == uint32(i) && en.ptr.ChkIdx == uint32(i) { found = true } } - c.Assert(found, IsTrue) + require.True(t, found) } // test some unexpected cases _, ok := m.Get(uint64(mod)) - c.Assert(ok, IsFalse) + require.False(t, ok) _, ok = m.Get(uint64(mod + 1)) - c.Assert(ok, IsFalse) + require.False(t, ok) } diff --git a/executor/distsql.go b/executor/distsql.go index 391dab353c647..899d2e822f176 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -198,7 +198,7 @@ type IndexReaderExecutor struct { // Close clears all resources hold by current object. func (e *IndexReaderExecutor) Close() error { - if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { return nil } @@ -210,7 +210,7 @@ func (e *IndexReaderExecutor) Close() 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 || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { req.Reset() return nil } @@ -262,13 +262,7 @@ func (e *IndexReaderExecutor) Open(ctx context.Context) error { return e.open(ctx, kvRanges) } -func isReadFromCache(tbl table.Table, stmt *stmtctx.StatementContext) bool { - if tbl != nil && tbl.Meta() != nil && tbl.Meta().TableCacheStatusType == model.TableCacheStatusEnable { - cond, _ := stmt.GetCacheTable(tbl.Meta().ID) - return cond - } - return false -} + func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) error { var err error if e.corColInFilter { @@ -285,9 +279,8 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) e.kvRanges = kvRanges // Treat temporary table as dummy table, avoid sending distsql request to TiKV. // In a test case IndexReaderExecutor is mocked and e.table is nil. - // Cache table is similar with temporary table, if it satisfies the read condition. // Avoid sending distsql request to TIKV. - if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { return nil } @@ -411,8 +404,8 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { return err } - // Treat temporary table as dummy table, avoid sending distsql request to TiKV. Cache table is similar with temporary table. - if e.table.Meta().TempTableType != model.TempTableNone || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + // Treat temporary table as dummy table, avoid sending distsql request to TiKV. + if e.table.Meta().TempTableType != model.TempTableNone { return nil } @@ -675,7 +668,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 || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table.Meta().TempTableType != model.TempTableNone { return nil } @@ -699,7 +692,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 || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table.Meta().TempTableType != model.TempTableNone { req.Reset() return nil } diff --git a/executor/executor.go b/executor/executor.go index 572f30649a6ad..7383941acd046 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1682,6 +1682,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.CTEStorageMap = map[int]*CTEStorages{} sc.IsStaleness = false sc.LockTableIDs = make(map[int64]struct{}) + sc.EnableOptimizeTrace = false + sc.LogicalOptimizeTrace = nil sc.InitMemTracker(memory.LabelForSQLText, vars.MemQuotaQuery) sc.InitDiskTracker(memory.LabelForSQLText, -1) diff --git a/executor/grant.go b/executor/grant.go index bf4582952f380..00cbee41123df 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -155,7 +155,7 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error { } _, err := internalSession.(sqlexec.SQLExecutor).ExecuteInternal(ctx, `INSERT INTO %n.%n (Host, User, authentication_string, plugin) VALUES (%?, %?, %?, %?);`, - mysql.SystemDB, mysql.UserTable, user.User.Hostname, user.User.Username, pwd, authPlugin) + mysql.SystemDB, mysql.UserTable, strings.ToLower(user.User.Hostname), user.User.Username, pwd, authPlugin) if err != nil { return err } @@ -476,7 +476,7 @@ func (e *GrantExec) grantGlobalLevel(priv *ast.PrivElem, user *ast.UserSpec, int if err != nil { return err } - sqlexec.MustFormatSQL(sql, ` WHERE User=%? AND Host=%?`, user.User.Username, user.User.Hostname) + sqlexec.MustFormatSQL(sql, ` WHERE User=%? AND Host=%?`, user.User.Username, strings.ToLower(user.User.Hostname)) _, err = internalSession.(sqlexec.SQLExecutor).ExecuteInternal(context.Background(), sql.String()) return err diff --git a/executor/insert.go b/executor/insert.go index ea4ba06dacdcd..714964c4715c8 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -389,6 +389,9 @@ func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRo // Update old row when the key is duplicated. e.evalBuffer4Dup.SetDatums(e.row4Update...) for _, col := range cols { + if col.LazyErr != nil { + return col.LazyErr + } val, err1 := col.Expr.Eval(e.evalBuffer4Dup.ToRow()) if err1 != nil { return err1 diff --git a/executor/mem_reader.go b/executor/mem_reader.go index 83901d35e58b2..08509832c0ca2 100644 --- a/executor/mem_reader.go +++ b/executor/mem_reader.go @@ -46,6 +46,7 @@ type memIndexReader struct { outputOffset []int // belowHandleCols is the handle's position of the below scan plan. belowHandleCols plannercore.HandleCols + cacheTable kv.MemBuffer } func buildMemIndexReader(us *UnionScanExec, idxReader *IndexReaderExecutor) *memIndexReader { @@ -64,6 +65,7 @@ func buildMemIndexReader(us *UnionScanExec, idxReader *IndexReaderExecutor) *mem retFieldTypes: retTypes(us), outputOffset: outputOffset, belowHandleCols: us.belowHandleCols, + cacheTable: us.cacheTable, } } @@ -92,7 +94,7 @@ func (m *memIndexReader) getMemRows() ([][]types.Datum, error) { } mutableRow := chunk.MutRowFromTypes(m.retFieldTypes) - err := iterTxnMemBuffer(m.table.ID, m.ctx, m.kvRanges, func(key, value []byte) error { + err := iterTxnMemBuffer(m.ctx, m.cacheTable, m.kvRanges, func(key, value []byte) error { data, err := m.decodeIndexKeyValue(key, value, tps) if err != nil { return err @@ -152,6 +154,7 @@ type memTableReader struct { colIDs map[int64]int buffer allocBuf pkColIDs []int64 + cacheTable kv.MemBuffer } type allocBuf struct { @@ -194,14 +197,15 @@ func buildMemTableReader(us *UnionScanExec, tblReader *TableReaderExecutor) *mem handleBytes: make([]byte, 0, 16), rd: rd, }, - pkColIDs: pkColIDs, + pkColIDs: pkColIDs, + cacheTable: us.cacheTable, } } // TODO: Try to make memXXXReader lazy, There is no need to decode many rows when parent operator only need 1 row. func (m *memTableReader) getMemRows() ([][]types.Datum, error) { mutableRow := chunk.MutRowFromTypes(m.retFieldTypes) - err := iterTxnMemBuffer(m.table.ID, m.ctx, m.kvRanges, func(key, value []byte) error { + err := iterTxnMemBuffer(m.ctx, m.cacheTable, m.kvRanges, func(key, value []byte) error { row, err := m.decodeRecordKeyValue(key, value) if err != nil { return err @@ -319,7 +323,7 @@ func hasColVal(data [][]byte, colIDs map[int64]int, id int64) bool { type processKVFunc func(key, value []byte) error -func iterTxnMemBuffer(tblID int64, ctx sessionctx.Context, kvRanges []kv.KeyRange, fn processKVFunc) error { +func iterTxnMemBuffer(ctx sessionctx.Context, cacheTable kv.MemBuffer, kvRanges []kv.KeyRange, fn processKVFunc) error { txn, err := ctx.Txn(true) if err != nil { return err @@ -327,10 +331,16 @@ func iterTxnMemBuffer(tblID int64, ctx sessionctx.Context, kvRanges []kv.KeyRang for _, rg := range kvRanges { iter := txn.GetMemBuffer().SnapshotIter(rg.StartKey, rg.EndKey) - iter, err = getMemIter(tblID, ctx, iter, rg) + snapCacheIter, err := getSnapIter(ctx, cacheTable, rg) if err != nil { return err } + if snapCacheIter != nil { + iter, err = transaction.NewUnionIter(iter, snapCacheIter, false) + if err != nil { + return err + } + } for ; iter.Valid(); err = iter.Next() { if err != nil { return err @@ -348,7 +358,7 @@ func iterTxnMemBuffer(tblID int64, ctx sessionctx.Context, kvRanges []kv.KeyRang return nil } -func getMemIter(tblID int64, ctx sessionctx.Context, iter kv.Iterator, rg kv.KeyRange) (kv.Iterator, error) { +func getSnapIter(ctx sessionctx.Context, cacheTable kv.MemBuffer, rg kv.KeyRange) (kv.Iterator, error) { var snapCacheIter kv.Iterator tempTableData := ctx.GetSessionVars().TemporaryTableData if tempTableData != nil { @@ -357,25 +367,14 @@ func getMemIter(tblID int64, ctx sessionctx.Context, iter kv.Iterator, rg kv.Key return nil, err } snapCacheIter = snapIter - } - cond, buffer := ctx.GetSessionVars().StmtCtx.GetCacheTable(tblID) - if cond { - cacheIter, err := buffer.(kv.MemBuffer).Iter(rg.StartKey, rg.EndKey) + } else if cacheTable != nil { + cacheIter, err := cacheTable.Iter(rg.StartKey, rg.EndKey) if err != nil { return nil, errors.Trace(err) } snapCacheIter = cacheIter } - - if snapCacheIter == nil { - return iter, nil - } - - newIter, err := transaction.NewUnionIter(iter, snapCacheIter, false) - if err != nil { - return nil, err - } - return newIter, nil + return snapCacheIter, nil } func reverseDatumSlice(rows [][]types.Datum) { @@ -386,7 +385,7 @@ func reverseDatumSlice(rows [][]types.Datum) { func (m *memIndexReader) getMemRowsHandle() ([]kv.Handle, error) { handles := make([]kv.Handle, 0, m.addedRowsLen) - err := iterTxnMemBuffer(m.table.ID, m.ctx, m.kvRanges, func(key, value []byte) error { + err := iterTxnMemBuffer(m.ctx, m.cacheTable, m.kvRanges, func(key, value []byte) error { handle, err := tablecodec.DecodeIndexHandle(key, value, len(m.index.Columns)) if err != nil { return err @@ -421,6 +420,8 @@ type memIndexLookUpReader struct { partitionMode bool // if it is accessing a partition table partitionTables []table.PhysicalTable // partition tables to access partitionKVRanges [][]kv.KeyRange // kv ranges for these partition tables + + cacheTable kv.MemBuffer } func buildMemIndexLookUpReader(us *UnionScanExec, idxLookUpReader *IndexLookUpExecutor) *memIndexLookUpReader { @@ -435,6 +436,7 @@ func buildMemIndexLookUpReader(us *UnionScanExec, idxLookUpReader *IndexLookUpEx retFieldTypes: retTypes(us), outputOffset: outputOffset, belowHandleCols: us.belowHandleCols, + cacheTable: us.cacheTable, } return &memIndexLookUpReader{ @@ -450,6 +452,7 @@ func buildMemIndexLookUpReader(us *UnionScanExec, idxLookUpReader *IndexLookUpEx partitionMode: idxLookUpReader.partitionTableMode, partitionKVRanges: idxLookUpReader.partitionKVRanges, partitionTables: idxLookUpReader.prunedPartitions, + cacheTable: us.cacheTable, } } @@ -517,6 +520,7 @@ func (m *memIndexLookUpReader) getMemRows() ([][]types.Datum, error) { handleBytes: make([]byte, 0, 16), rd: rd, }, + cacheTable: m.cacheTable, } return memTblReader.getMemRows() diff --git a/executor/point_get.go b/executor/point_get.go index 0d1de79d0d1cd..489bbf9bb8085 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -58,6 +58,10 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, } + + if p.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { + e.cacheTable = b.getCacheTable(p.TblInfo, startTS) + } e.base().initCap = 1 e.base().maxChunkSize = 1 e.Init(p, startTS) @@ -96,7 +100,8 @@ type PointGetExecutor struct { // virtualColumnRetFieldTypes records the RetFieldTypes of virtual columns. virtualColumnRetFieldTypes []*types.FieldType - stats *runtimeStatsWithSnapshot + stats *runtimeStatsWithSnapshot + cacheTable kv.MemBuffer } // Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field @@ -150,6 +155,9 @@ func (e *PointGetExecutor) Open(context.Context) error { } else { e.snapshot = e.ctx.GetSnapshotWithTS(snapshotTS) } + if e.cacheTable != nil { + e.snapshot = cacheTableSnapshot{e.snapshot, e.cacheTable} + } if err := e.verifyTxnScope(); err != nil { return err } diff --git a/executor/revoke.go b/executor/revoke.go index ac279e65828b2..5a66cf3e405ee 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -216,7 +216,7 @@ func (e *RevokeExec) revokeGlobalPriv(internalSession sessionctx.Context, priv * if err != nil { return err } - sqlexec.MustFormatSQL(sql, " WHERE User=%? AND Host=%?", user, host) + sqlexec.MustFormatSQL(sql, " WHERE User=%? AND Host=%?", user, strings.ToLower(host)) _, err = internalSession.(sqlexec.SQLExecutor).ExecuteInternal(context.Background(), sql.String()) return err diff --git a/executor/set.go b/executor/set.go index 5ad8790a746f3..65e21d470cc78 100644 --- a/executor/set.go +++ b/executor/set.go @@ -124,8 +124,8 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres if err != nil { return err } - // Some PD client dynamic options need to be checked and set here. - err = e.checkPDClientDynamicOption(name, valStr, sessionVars) + // Some PD client dynamic options need to be checked first and set here. + err = e.checkPDClientDynamicOption(name, sessionVars) if err != nil { return err } @@ -204,16 +204,30 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres return nil } -func (e *SetExecutor) checkPDClientDynamicOption(name, valStr string, sessionVars *variable.SessionVars) error { - var err error +func (e *SetExecutor) checkPDClientDynamicOption(name string, sessionVars *variable.SessionVars) error { + if name != variable.TiDBTSOClientBatchMaxWaitTime && + name != variable.TiDBEnableTSOFollowerProxy { + return nil + } + var ( + err error + valStr string + ) + valStr, err = sessionVars.GlobalVarsAccessor.GetGlobalSysVar(name) + if err != nil { + return err + } switch name { case variable.TiDBTSOClientBatchMaxWaitTime: - var val int64 - val, err = strconv.ParseInt(valStr, 10, 64) + var val float64 + val, err = strconv.ParseFloat(valStr, 64) if err != nil { return err } - err = domain.GetDomain(e.ctx).SetPDClientDynamicOption(pd.MaxTSOBatchWaitInterval, time.Millisecond*time.Duration(val)) + err = domain.GetDomain(e.ctx).SetPDClientDynamicOption( + pd.MaxTSOBatchWaitInterval, + time.Duration(float64(time.Millisecond)*val), + ) if err != nil { return err } diff --git a/executor/set_test.go b/executor/set_test.go index d53cd64b20966..2595b4a3131f5 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -559,14 +559,26 @@ func (s *testSerialSuite1) TestSetVar(c *C) { tk.MustQuery("select @@session.tidb_opt_prefer_range_scan").Check(testkit.Rows("0")) tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 0.5") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("0.5")) tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 1") tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("1")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 1.5") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("1.5")) tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 10") tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("10")) tk.MustExec("set global tidb_tso_client_batch_max_wait_time = -1") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_tso_client_batch_max_wait_time value: '-1'")) - tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 11") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_tso_client_batch_max_wait_time value: '11'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = -0.01") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_tso_client_batch_max_wait_time value: '-0.01'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 10.01") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_tso_client_batch_max_wait_time value: '10.01'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("10")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 10.1") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_tso_client_batch_max_wait_time value: '10.1'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time").Check(testkit.Rows("10")) c.Assert(tk.ExecToErr("set tidb_tso_client_batch_max_wait_time = 1"), NotNil) tk.MustQuery("select @@tidb_enable_tso_follower_proxy").Check(testkit.Rows("0")) diff --git a/executor/show.go b/executor/show.go index 0dfbf98843f3a..9fcbb67e7cf66 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1433,7 +1433,7 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error { exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, `SELECT plugin FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, hostName) + stmt, err := exec.ParseWithParams(ctx, `SELECT plugin FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, strings.ToLower(hostName)) if err != nil { return errors.Trace(err) } diff --git a/executor/simple.go b/executor/simple.go index f42de4b927e96..519c02a37834b 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -944,7 +944,7 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) } stmt, err := exec.ParseWithParams(ctx, `UPDATE %n.%n SET authentication_string=%?, plugin=%? WHERE Host=%? and User=%?;`, - mysql.SystemDB, mysql.UserTable, pwd, spec.AuthOpt.AuthPlugin, spec.User.Hostname, spec.User.Username, + mysql.SystemDB, mysql.UserTable, pwd, spec.AuthOpt.AuthPlugin, strings.ToLower(spec.User.Hostname), spec.User.Username, ) if err != nil { return err @@ -1158,7 +1158,7 @@ func renameUserHostInSystemTable(sqlExecutor sqlexec.SQLExecutor, tableName, use sqlexec.MustFormatSQL(sql, `UPDATE %n.%n SET %n = %?, %n = %? WHERE %n = %? and %n = %?;`, mysql.SystemDB, tableName, usernameColumn, users.NewUser.Username, hostColumn, strings.ToLower(users.NewUser.Hostname), - usernameColumn, users.OldUser.Username, hostColumn, users.OldUser.Hostname) + usernameColumn, users.OldUser.Username, hostColumn, strings.ToLower(users.OldUser.Hostname)) _, err := sqlExecutor.ExecuteInternal(context.TODO(), sql.String()) return err } @@ -1225,7 +1225,7 @@ func (e *SimpleExec) executeDropUser(ctx context.Context, s *ast.DropUserStmt) e // begin a transaction to delete a user. sql.Reset() - sqlexec.MustFormatSQL(sql, `DELETE FROM %n.%n WHERE Host = %? and User = %?;`, mysql.SystemDB, mysql.UserTable, user.Hostname, user.Username) + sqlexec.MustFormatSQL(sql, `DELETE FROM %n.%n WHERE Host = %? and User = %?;`, mysql.SystemDB, mysql.UserTable, strings.ToLower(user.Hostname), user.Username) if _, err = sqlExecutor.ExecuteInternal(context.TODO(), sql.String()); err != nil { failedUsers = append(failedUsers, user.String()) break @@ -1417,7 +1417,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, `UPDATE %n.%n SET authentication_string=%? WHERE User=%? AND Host=%?;`, mysql.SystemDB, mysql.UserTable, pwd, u, h) + stmt, err := exec.ParseWithParams(ctx, `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 47359392d7a1c..271b51d591008 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -523,6 +523,12 @@ func (s *testSuite7) TestUser(c *C) { Check(testkit.Rows("engineering india")) tk.MustQuery("select user,host from mysql.user where user='engineering' and host = 'us'"). Check(testkit.Rows("engineering us")) + + tk.MustExec("drop role engineering@INDIA;") + tk.MustExec("drop role engineering@US;") + + tk.MustQuery("select user from mysql.user where user='engineering' and host = 'india'").Check(testkit.Rows()) + tk.MustQuery("select user from mysql.user where user='engineering' and host = 'us'").Check(testkit.Rows()) } func (s *testSuite3) TestSetPwd(c *C) { diff --git a/executor/table_reader.go b/executor/table_reader.go index 88b7c1751ccdd..958b8cc442061 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -180,7 +180,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) || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { kvReq, err := e.buildKVReq(ctx, firstPartRanges) if err != nil { return err @@ -218,7 +218,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) || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { // Treat temporary table as dummy table, avoid sending distsql request to TiKV. req.Reset() return nil @@ -263,7 +263,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) || isReadFromCache(e.table, e.ctx.GetSessionVars().StmtCtx) { + if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { return nil } diff --git a/executor/union_scan.go b/executor/union_scan.go index e08e90249375c..c796d36bb6d31 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -56,6 +56,9 @@ type UnionScanExec struct { // virtualColumnIndex records all the indices of virtual columns and sort them in definition // to make sure we can compute the virtual column in right order. virtualColumnIndex []int + + // cacheTable not nil means it's reading from cached table. + cacheTable kv.MemBuffer } // Open implements the Executor Open interface. @@ -202,6 +205,10 @@ func (us *UnionScanExec) getOneRow(ctx context.Context) ([]types.Datum, error) { } func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, error) { + if us.cacheTable != nil { + // From cache table, so the snapshot is nil + return nil, nil + } if us.cursor4SnapshotRows < len(us.snapshotRows) { return us.snapshotRows[us.cursor4SnapshotRows], nil } diff --git a/executor/utils_test.go b/executor/utils_test.go index 2cc64849ca922..53301fb255a00 100644 --- a/executor/utils_test.go +++ b/executor/utils_test.go @@ -15,11 +15,15 @@ package executor import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/errors" + "github.com/stretchr/testify/require" ) -func (s *pkgTestSuite) TestBatchRetrieverHelper(c *C) { +func TestBatchRetrieverHelper(t *testing.T) { + t.Parallel() + rangeStarts := make([]int, 0) rangeEnds := make([]int, 0) collect := func(start, end int) error { @@ -30,9 +34,9 @@ func (s *pkgTestSuite) TestBatchRetrieverHelper(c *C) { r := &batchRetrieverHelper{} err := r.nextBatch(collect) - c.Assert(err, IsNil) - c.Assert(rangeStarts, DeepEquals, []int{}) - c.Assert(rangeEnds, DeepEquals, []int{}) + require.NoError(t, err) + require.Equal(t, rangeStarts, []int{}) + require.Equal(t, rangeEnds, []int{}) r = &batchRetrieverHelper{ retrieved: true, @@ -40,9 +44,9 @@ func (s *pkgTestSuite) TestBatchRetrieverHelper(c *C) { totalRows: 10, } err = r.nextBatch(collect) - c.Assert(err, IsNil) - c.Assert(rangeStarts, DeepEquals, []int{}) - c.Assert(rangeEnds, DeepEquals, []int{}) + require.NoError(t, err) + require.Equal(t, rangeStarts, []int{}) + require.Equal(t, rangeEnds, []int{}) r = &batchRetrieverHelper{ batchSize: 3, @@ -51,8 +55,8 @@ func (s *pkgTestSuite) TestBatchRetrieverHelper(c *C) { err = r.nextBatch(func(start, end int) error { return errors.New("some error") }) - c.Assert(err, NotNil) - c.Assert(r.retrieved, IsTrue) + require.Error(t, err) + require.True(t, r.retrieved) r = &batchRetrieverHelper{ batchSize: 3, @@ -60,10 +64,10 @@ func (s *pkgTestSuite) TestBatchRetrieverHelper(c *C) { } for !r.retrieved { err = r.nextBatch(collect) - c.Assert(err, IsNil) + require.NoError(t, err) } - c.Assert(rangeStarts, DeepEquals, []int{0, 3, 6, 9}) - c.Assert(rangeEnds, DeepEquals, []int{3, 6, 9, 10}) + require.Equal(t, rangeStarts, []int{0, 3, 6, 9}) + require.Equal(t, rangeEnds, []int{3, 6, 9, 10}) rangeStarts = rangeStarts[:0] rangeEnds = rangeEnds[:0] @@ -73,10 +77,10 @@ func (s *pkgTestSuite) TestBatchRetrieverHelper(c *C) { } for !r.retrieved { err = r.nextBatch(collect) - c.Assert(err, IsNil) + require.NoError(t, err) } - c.Assert(rangeStarts, DeepEquals, []int{0, 3, 6}) - c.Assert(rangeEnds, DeepEquals, []int{3, 6, 9}) + require.Equal(t, rangeStarts, []int{0, 3, 6}) + require.Equal(t, rangeEnds, []int{3, 6, 9}) rangeStarts = rangeStarts[:0] rangeEnds = rangeEnds[:0] @@ -86,8 +90,8 @@ func (s *pkgTestSuite) TestBatchRetrieverHelper(c *C) { } for !r.retrieved { err = r.nextBatch(collect) - c.Assert(err, IsNil) + require.NoError(t, err) } - c.Assert(rangeStarts, DeepEquals, []int{0}) - c.Assert(rangeEnds, DeepEquals, []int{10}) + require.Equal(t, rangeStarts, []int{0}) + require.Equal(t, rangeEnds, []int{10}) } diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index b155370d64462..407982af11e31 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -1931,7 +1931,7 @@ func WrapWithCastAsString(ctx sessionctx.Context, expr Expression) Expression { } tp := types.NewFieldType(mysql.TypeVarString) if expr.Coercibility() == CoercibilityExplicit { - tp.Charset, tp.Collate = expr.CharsetAndCollation(ctx) + tp.Charset, tp.Collate = expr.CharsetAndCollation() } else { tp.Charset, tp.Collate = ctx.GetSessionVars().GetCharsetInfo() } diff --git a/expression/builtin_encryption.go b/expression/builtin_encryption.go index f6badf0fcc72c..75dfa7d179561 100644 --- a/expression/builtin_encryption.go +++ b/expression/builtin_encryption.go @@ -159,10 +159,18 @@ func (b *builtinAesDecryptSig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", true, err } + cryptStr, err = charset.NewEncoding(b.args[0].GetType().Charset).EncodeString(cryptStr) + if err != nil { + return "", false, err + } keyStr, isNull, err := b.args[1].EvalString(b.ctx, row) if isNull || err != nil { return "", true, err } + keyStr, err = charset.NewEncoding(b.args[1].GetType().Charset).EncodeString(keyStr) + if err != nil { + return "", false, err + } if !b.ivRequired && len(b.args) == 3 { // For modes that do not require init_vector, it is ignored and a warning is generated if it is specified. b.ctx.GetSessionVars().StmtCtx.AppendWarning(errWarnOptionIgnored.GenWithStackByArgs("IV")) @@ -202,16 +210,28 @@ func (b *builtinAesDecryptIVSig) evalString(row chunk.Row) (string, bool, error) if isNull || err != nil { return "", true, err } + cryptStr, err = charset.NewEncoding(b.args[0].GetType().Charset).EncodeString(cryptStr) + if err != nil { + return "", false, err + } keyStr, isNull, err := b.args[1].EvalString(b.ctx, row) if isNull || err != nil { return "", true, err } + keyStr, err = charset.NewEncoding(b.args[1].GetType().Charset).EncodeString(keyStr) + if err != nil { + return "", false, err + } iv, isNull, err := b.args[2].EvalString(b.ctx, row) if isNull || err != nil { return "", true, err } + iv, err = charset.NewEncoding(b.args[2].GetType().Charset).EncodeString(iv) + if err != nil { + return "", false, err + } if len(iv) < aes.BlockSize { return "", true, errIncorrectArgs.GenWithStack("The initialization vector supplied to aes_decrypt is too short. Must be at least %d bytes long", aes.BlockSize) } @@ -856,6 +876,11 @@ func (b *builtinCompressSig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", true, err } + strTp := b.args[0].GetType() + str, err = charset.NewEncoding(strTp.Charset).EncodeString(str) + if err != nil { + return "", false, err + } // According to doc: Empty strings are stored as empty strings. if len(str) == 0 { diff --git a/expression/builtin_encryption_test.go b/expression/builtin_encryption_test.go index 40e1d19f95d07..d677019754d7c 100644 --- a/expression/builtin_encryption_test.go +++ b/expression/builtin_encryption_test.go @@ -192,6 +192,47 @@ func TestAESDecrypt(t *testing.T) { require.NoError(t, err) testNullInput(t, ctx, ast.AesDecrypt) testAmbiguousInput(t, ctx, ast.AesDecrypt) + + // Test GBK String + gbkStr, _ := charset.NewEncoding("gbk").EncodeString("你好") + gbkTests := []struct { + mode string + chs string + origin interface{} + params []interface{} + crypt string + }{ + // test for ecb + {"aes-128-ecb", "utf8mb4", "你好", []interface{}{"123"}, "CEBD80EEC6423BEAFA1BB30FD7625CBC"}, + {"aes-128-ecb", "gbk", gbkStr, []interface{}{"123"}, "6AFA9D7BA2C1AED1603E804F75BB0127"}, + {"aes-128-ecb", "utf8mb4", "123", []interface{}{"你好"}, "E03F6D9C1C86B82F5620EE0AA9BD2F6A"}, + {"aes-128-ecb", "gbk", "123", []interface{}{gbkStr}, "31A2D26529F0E6A38D406379ABD26FA5"}, + {"aes-128-ecb", "utf8mb4", "你好", []interface{}{"你好"}, "3E2D8211DAE17143F22C2C5969A35263"}, + {"aes-128-ecb", "gbk", gbkStr, []interface{}{gbkStr}, "84982910338160D037615D283AD413DE"}, + // test for cbc + {"aes-128-cbc", "utf8mb4", "你好", []interface{}{"123", "1234567890123456"}, "B95509A516ACED59C3DF4EC41C538D83"}, + {"aes-128-cbc", "gbk", gbkStr, []interface{}{"123", "1234567890123456"}, "D4322D091B5DDE0DEB35B1749DA2483C"}, + {"aes-128-cbc", "utf8mb4", "123", []interface{}{"你好", "1234567890123456"}, "E19E86A9E78E523267AFF36261AD117D"}, + {"aes-128-cbc", "gbk", "123", []interface{}{gbkStr, "1234567890123456"}, "5A2F8F2C1841CC4E1D1640F1EA2A1A23"}, + {"aes-128-cbc", "utf8mb4", "你好", []interface{}{"你好", "1234567890123456"}, "B73637C73302C909EA63274C07883E71"}, + {"aes-128-cbc", "gbk", gbkStr, []interface{}{gbkStr, "1234567890123456"}, "61E13E9B00F2E757F4E925D3268227A0"}, + } + + for _, tt := range gbkTests { + err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, tt.chs) + require.NoError(t, err) + err = variable.SetSessionSystemVar(ctx.GetSessionVars(), variable.BlockEncryptionMode, tt.mode) + require.NoError(t, err) + args := []types.Datum{fromHex(tt.crypt)} + for _, param := range tt.params { + args = append(args, types.NewDatum(param)) + } + f, err := fc.getFunction(ctx, datumsToConstants(args)) + require.NoError(t, err) + str, err := evalBuiltinFunc(f, chunk.Row{}) + require.NoError(t, err) + require.Equal(t, types.NewCollationStringDatum(tt.origin.(string), charset.CollationBin), str) + } } func testNullInput(t *testing.T, ctx sessionctx.Context, fnName string) { @@ -462,17 +503,24 @@ func decodeHex(str string) []byte { func TestCompress(t *testing.T) { t.Parallel() ctx := createContext(t) + fc := funcs[ast.Compress] + gbkStr, _ := charset.NewEncoding("gbk").EncodeString("你好") tests := []struct { + chs string in interface{} expect interface{} }{ - {"hello world", string(decodeHex("0B000000789CCA48CDC9C95728CF2FCA4901040000FFFF1A0B045D"))}, - {"", ""}, - {nil, nil}, + {"", "hello world", string(decodeHex("0B000000789CCA48CDC9C95728CF2FCA4901040000FFFF1A0B045D"))}, + {"", "", ""}, + {"", nil, nil}, + {"utf8mb4", "hello world", string(decodeHex("0B000000789CCA48CDC9C95728CF2FCA4901040000FFFF1A0B045D"))}, + {"gbk", "hello world", string(decodeHex("0B000000789CCA48CDC9C95728CF2FCA4901040000FFFF1A0B045D"))}, + {"utf8mb4", "你好", string(decodeHex("06000000789C7AB277C1D3A57B01010000FFFF10450489"))}, + {"gbk", gbkStr, string(decodeHex("04000000789C3AF278D76140000000FFFF07F40325"))}, } - - fc := funcs[ast.Compress] for _, test := range tests { + err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, test.chs) + require.NoErrorf(t, err, "%v", test) arg := types.NewDatum(test.in) f, err := fc.getFunction(ctx, datumsToConstants([]types.Datum{arg})) require.NoErrorf(t, err, "%v", test) diff --git a/expression/builtin_encryption_vec.go b/expression/builtin_encryption_vec.go index 3b6c0b5cffd4d..9d076a3234232 100644 --- a/expression/builtin_encryption_vec.go +++ b/expression/builtin_encryption_vec.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/encrypt" - "github.com/pingcap/tidb/util/hack" ) func (b *builtinAesDecryptSig) vectorized() bool { @@ -68,9 +67,19 @@ func (b *builtinAesDecryptSig) vecEvalString(input *chunk.Chunk, result *chunk.C isWarning := !b.ivRequired && len(b.args) == 3 isConstKey := b.args[1].ConstItem(b.ctx.GetSessionVars().StmtCtx) - var key []byte + cryptEnc := charset.NewEncoding(b.args[0].GetType().Charset) + keyEnc := charset.NewEncoding(b.args[1].GetType().Charset) + + var ( + key []byte + encodedBuf []byte + ) if isConstKey { - key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize) + keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(0)) + if err != nil { + return err + } + key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) } result.ReserveString(n) @@ -86,12 +95,19 @@ func (b *builtinAesDecryptSig) vecEvalString(input *chunk.Chunk, result *chunk.C stmtCtx.AppendWarning(errWarnOptionIgnored.GenWithStackByArgs("IV")) } if !isConstKey { - key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(i), b.keySize) + keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(i)) + if err != nil { + return err + } + key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) } // ANNOTATION: // we can't use GetBytes here because GetBytes return raw memory in strBuf, // and the memory will be modified in AESEncryptWithECB & AESDecryptWithECB - str := []byte(strBuf.GetString(i)) + str, err := cryptEnc.Encode(encodedBuf, []byte(strBuf.GetString(i))) + if err != nil { + return err + } plainText, err := encrypt.AESDecryptWithECB(str, key) if err != nil { result.AppendNull() @@ -333,6 +349,10 @@ func (b *builtinAesDecryptIVSig) vecEvalString(input *chunk.Chunk, result *chunk return err } + cryptEnc := charset.NewEncoding(b.args[0].GetType().Charset) + keyEnc := charset.NewEncoding(b.args[1].GetType().Charset) + ivEnc := charset.NewEncoding(b.args[2].GetType().Charset) + isCBC := false isOFB := false isCFB := false @@ -348,9 +368,19 @@ func (b *builtinAesDecryptIVSig) vecEvalString(input *chunk.Chunk, result *chunk } isConst := b.args[1].ConstItem(b.ctx.GetSessionVars().StmtCtx) - var key []byte + var ( + key []byte + // key and str can share the buf as DeriveKeyMySQL returns new byte slice + // iv needs a spare buf as it works on the buf directly + encodedBuf []byte + ivEncodedBuf []byte + ) if isConst { - key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize) + keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(0)) + if err != nil { + return err + } + key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) } result.ReserveString(n) @@ -361,28 +391,39 @@ func (b *builtinAesDecryptIVSig) vecEvalString(input *chunk.Chunk, result *chunk continue } - iv := ivBuf.GetBytes(i) + iv, err := ivEnc.Encode(ivEncodedBuf, ivBuf.GetBytes(i)) + if err != nil { + return err + } if len(iv) < aes.BlockSize { return errIncorrectArgs.GenWithStack("The initialization vector supplied to aes_decrypt is too short. Must be at least %d bytes long", aes.BlockSize) } // init_vector must be 16 bytes or longer (bytes in excess of 16 are ignored) iv = iv[0:aes.BlockSize] if !isConst { - key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(i), b.keySize) + keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(i)) + if err != nil { + return err + } + key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) } var plainText []byte // ANNOTATION: // we can't use GetBytes here because GetBytes return raw memory in strBuf, // and the memory will be modified in AESDecryptWithCBC & AESDecryptWithOFB & AESDecryptWithCFB + str, err := cryptEnc.Encode(encodedBuf, []byte(strBuf.GetString(i))) + if err != nil { + return err + } if isCBC { - plainText, err = encrypt.AESDecryptWithCBC([]byte(strBuf.GetString(i)), key, iv) + plainText, err = encrypt.AESDecryptWithCBC(str, key, iv) } if isOFB { - plainText, err = encrypt.AESDecryptWithOFB([]byte(strBuf.GetString(i)), key, iv) + plainText, err = encrypt.AESDecryptWithOFB(str, key, iv) } if isCFB { - plainText, err = encrypt.AESDecryptWithCFB([]byte(strBuf.GetString(i)), key, iv) + plainText, err = encrypt.AESDecryptWithCFB(str, key, iv) } if err != nil { result.AppendNull() @@ -584,6 +625,9 @@ func (b *builtinCompressSig) vecEvalString(input *chunk.Chunk, result *chunk.Col if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } + bufTp := b.args[0].GetType() + bufEnc := charset.NewEncoding(bufTp.Charset) + var encodedBuf []byte result.ReserveString(n) for i := 0; i < n; i++ { @@ -592,14 +636,19 @@ func (b *builtinCompressSig) vecEvalString(input *chunk.Chunk, result *chunk.Col continue } - str := buf.GetString(i) + str := buf.GetBytes(i) // According to doc: Empty strings are stored as empty strings. if len(str) == 0 { result.AppendString("") } - compressed, err := deflate(hack.Slice(str)) + strBuf, err := bufEnc.Encode(encodedBuf, str) + if err != nil { + return err + } + + compressed, err := deflate(strBuf) if err != nil { result.AppendNull() continue @@ -617,7 +666,7 @@ func (b *builtinCompressSig) vecEvalString(input *chunk.Chunk, result *chunk.Col defer deallocateByteSlice(buffer) buffer = buffer[:resultLength] - binary.LittleEndian.PutUint32(buffer, uint32(len(str))) + binary.LittleEndian.PutUint32(buffer, uint32(len(strBuf))) copy(buffer[4:], compressed) if shouldAppendSuffix { diff --git a/expression/builtin_string.go b/expression/builtin_string.go index b5b495321e16c..13733abd4cc10 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -2446,8 +2446,14 @@ func (b *builtinCharSig) evalString(row chunk.Row) (string, bool, error) { } bigints = append(bigints, val) } - result := string(b.convertToBytes(bigints)) - return result, false, nil + + dBytes := b.convertToBytes(bigints) + resultBytes, err := charset.NewEncoding(b.tp.Charset).Decode(nil, dBytes) + if err != nil { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(err) + return "", true, nil + } + return string(resultBytes), false, nil } type charLengthFunctionClass struct { @@ -3629,10 +3635,9 @@ func (b *builtinToBase64Sig) evalString(row chunk.Row) (d string, isNull bool, e return "", isNull, err } argTp := b.args[0].GetType() - if !types.IsBinaryStr(argTp) { - if encodedStr, err := charset.NewEncoding(argTp.Charset).EncodeString(str); err == nil { - str = encodedStr - } + str, err = charset.NewEncoding(argTp.Charset).EncodeString(str) + if err != nil { + return "", false, err } needEncodeLen := base64NeededEncodedLength(len(str)) if needEncodeLen == -1 { diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 6a98d367107e6..28d98d0215091 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -1430,34 +1430,37 @@ func TestChar(t *testing.T) { }() tbl := []struct { - str string - iNum int64 - fNum float64 - result string + str string + iNum int64 + fNum float64 + charset interface{} + result interface{} + warnings int }{ - {"65", 66, 67.5, "ABD"}, // float - {"65", 16740, 67.5, "AAdD"}, // large num - {"65", -1, 67.5, "A\xff\xff\xff\xffD"}, // nagtive int - {"a", -1, 67.5, "\x00\xff\xff\xff\xffD"}, // invalid 'a' + {"65", 66, 67.5, "utf8", "ABD", 0}, // float + {"65", 16740, 67.5, "utf8", "AAdD", 0}, // large num + {"65", -1, 67.5, nil, "A\xff\xff\xff\xffD", 0}, // nagtive int + {"a", -1, 67.5, nil, "\x00\xff\xff\xff\xffD", 0}, // invalid 'a' + // TODO: Uncomment it when issue #29685 be closed + // {"65", -1, 67.5, "utf8", nil, 1}, // with utf8, return nil + // {"a", -1, 67.5, "utf8", nil, 2}, // with utf8, return nil + // TODO: Uncomment it when gbk be added into charsetInfos + // {"1234567", 1234567, 1234567, "gbk", "謬謬謬", 0}, // test char for gbk + // {"123456789", 123456789, 123456789, "gbk", nil, 3}, // invalid 123456789 in gbk } for _, v := range tbl { - for _, char := range []interface{}{"utf8", nil} { - fc := funcs[ast.CharFunc] - f, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums(v.str, v.iNum, v.fNum, char))) - require.NoError(t, err) - require.NotNil(t, f) - r, err := evalBuiltinFunc(f, chunk.Row{}) - require.NoError(t, err) - trequire.DatumEqual(t, types.NewDatum(v.result), r) + fc := funcs[ast.CharFunc] + f, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums(v.str, v.iNum, v.fNum, v.charset))) + require.NoError(t, err) + require.NotNil(t, f) + r, err := evalBuiltinFunc(f, chunk.Row{}) + require.NoError(t, err) + trequire.DatumEqual(t, types.NewDatum(v.result), r) + if v.warnings != 0 { + warnings := ctx.GetSessionVars().StmtCtx.GetWarnings() + require.Equal(t, v.warnings, len(warnings)) } } - - fc := funcs[ast.CharFunc] - f, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums("65", 66, nil))) - require.NoError(t, err) - r, err := evalBuiltinFunc(f, chunk.Row{}) - require.NoError(t, err) - trequire.DatumEqual(t, types.NewDatum("AB"), r) } func TestCharLength(t *testing.T) { diff --git a/expression/builtin_string_vec.go b/expression/builtin_string_vec.go index 531d2379e1135..ca21e724f0dfa 100644 --- a/expression/builtin_string_vec.go +++ b/expression/builtin_string_vec.go @@ -2301,16 +2301,26 @@ func (b *builtinCharSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) for i := 0; i < l-1; i++ { bufint[i] = buf[i].Int64s() } + var resultBytes []byte + enc := charset.NewEncoding(b.tp.Charset) for i := 0; i < n; i++ { bigints = bigints[0:0] for j := 0; j < l-1; j++ { if buf[j].IsNull(i) { + result.AppendNull() continue } bigints = append(bigints, bufint[j][i]) } - tempString := string(b.convertToBytes(bigints)) - result.AppendString(tempString) + dBytes := b.convertToBytes(bigints) + + resultBytes, err := enc.Decode(resultBytes, dBytes) + if err != nil { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(err) + result.AppendNull() + continue + } + result.AppendString(string(resultBytes)) } return nil } @@ -2463,7 +2473,6 @@ func (b *builtinToBase64Sig) vecEvalString(input *chunk.Chunk, result *chunk.Col argTp := b.args[0].GetType() enc := charset.NewEncoding(argTp.Charset) - isBinaryStr := types.IsBinaryStr(argTp) result.ReserveString(n) for i := 0; i < n; i++ { @@ -2471,12 +2480,9 @@ func (b *builtinToBase64Sig) vecEvalString(input *chunk.Chunk, result *chunk.Col result.AppendNull() continue } - str := buf.GetString(i) - if !isBinaryStr { - str, err = enc.EncodeString(str) - if err != nil { - return err - } + str, err := enc.EncodeString(buf.GetString(i)) + if err != nil { + return err } needEncodeLen := base64NeededEncodedLength(len(str)) if needEncodeLen == -1 { diff --git a/expression/collation.go b/expression/collation.go index ed7d673c56d28..7db5645941601 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -68,17 +68,7 @@ func (c *collationInfo) SetCharsetAndCollation(chs, coll string) { c.charset, c.collation = chs, coll } -func (c *collationInfo) CharsetAndCollation(ctx sessionctx.Context) (string, string) { - if c.charset != "" || c.collation != "" { - return c.charset, c.collation - } - - if ctx != nil && ctx.GetSessionVars() != nil { - c.charset, c.collation = ctx.GetSessionVars().GetCharsetInfo() - } - if c.charset == "" || c.collation == "" { - c.charset, c.collation = charset.GetDefaultCharsetAndCollate() - } +func (c *collationInfo) CharsetAndCollation() (string, string) { return c.charset, c.collation } @@ -99,10 +89,10 @@ type CollationInfo interface { // SetRepertoire sets a specified repertoire for this expression. SetRepertoire(r Repertoire) - // CharsetAndCollation ... - CharsetAndCollation(ctx sessionctx.Context) (string, string) + // CharsetAndCollation gets charset and collation. + CharsetAndCollation() (string, string) - // SetCharsetAndCollation ... + // SetCharsetAndCollation sets charset and collation. SetCharsetAndCollation(chs, coll string) } diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index b007d4b79a2eb..70fea317fe57b 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -152,7 +152,7 @@ func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Exp } for idx, expr := range sf.GetArgs() { if src.Equal(nil, expr) { - _, coll := cond.CharsetAndCollation(ctx) + _, coll := cond.CharsetAndCollation() if tgt.GetType().Collate != coll { continue } diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 99693677091bc..d4f0d5eb84fd9 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -967,7 +967,7 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti case tipb.ScalarFuncSig_HexStrArg: chs, args := "utf-8", base.getArgs() if len(args) == 1 { - chs, _ = args[0].CharsetAndCollation(ctx) + chs, _ = args[0].CharsetAndCollation() } f = &builtinHexStrArgSig{base, charset.NewEncoding(chs)} case tipb.ScalarFuncSig_InsertUTF8: diff --git a/expression/errors.go b/expression/errors.go index 0d2714ad683ad..cfadcc6811e02 100644 --- a/expression/errors.go +++ b/expression/errors.go @@ -34,6 +34,8 @@ var ( ErrInvalidArgumentForLogarithm = dbterror.ClassExpression.NewStd(mysql.ErrInvalidArgumentForLogarithm) ErrIncorrectType = dbterror.ClassExpression.NewStd(mysql.ErrIncorrectType) ErrInvalidTableSample = dbterror.ClassExpression.NewStd(mysql.ErrInvalidTableSample) + ErrInternal = dbterror.ClassOptimizer.NewStd(mysql.ErrInternal) + ErrNoDB = dbterror.ClassOptimizer.NewStd(mysql.ErrNoDB) // All the un-exported errors are defined here: errFunctionNotExists = dbterror.ClassExpression.NewStd(mysql.ErrSpDoesNotExist) diff --git a/expression/expr_to_pb.go b/expression/expr_to_pb.go index 569b8450397d3..5cb01638eae0b 100644 --- a/expression/expr_to_pb.go +++ b/expression/expr_to_pb.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -38,8 +37,7 @@ func ExpressionsToPBList(sc *stmtctx.StatementContext, exprs []Expression, clien for _, expr := range exprs { v := pc.ExprToPB(expr) if v == nil { - return nil, dbterror.ClassOptimizer.NewStd(mysql.ErrInternal). - GenWithStack("expression %v cannot be pushed down", expr) + return nil, ErrInternal.GenWithStack("expression %v cannot be pushed down", expr) } pbExpr = append(pbExpr, v) } @@ -274,7 +272,7 @@ func (pc PbConverter) scalarFuncToPBExpr(expr *ScalarFunction) *tipb.Expr { // put collation information into the RetType enforcedly and push it down to TiKV/MockTiKV tp := *expr.RetType if collate.NewCollationEnabled() { - _, tp.Collate = expr.CharsetAndCollation(expr.GetCtx()) + _, tp.Collate = expr.CharsetAndCollation() } // Construct expression ProtoBuf. diff --git a/expression/expr_to_pb_serial_test.go b/expression/expr_to_pb_serial_test.go index d4e21165f688d..624e5dcb90a23 100644 --- a/expression/expr_to_pb_serial_test.go +++ b/expression/expr_to_pb_serial_test.go @@ -50,7 +50,7 @@ func TestPushCollationDown(t *testing.T) { require.NoError(t, err) expr, err := PBToExpr(pbExpr[0], tps, sc) require.NoError(t, err) - _, eColl := expr.CharsetAndCollation(nil) + _, eColl := expr.CharsetAndCollation() require.Equal(t, coll, eColl) } } diff --git a/expression/expression.go b/expression/expression.go index 6bf6320a19579..fda1615fd8b4e 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -748,6 +748,9 @@ type Assignment struct { // ColName indicates its original column name in table schema. It's used for outputting helping message when executing meets some errors. ColName model.CIStr Expr Expression + // LazyErr is used in statement like `INSERT INTO t1 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);`, ErrSubqueryMoreThan1Row + // should be evaluated after the duplicate situation is detected in the executing procedure. + LazyErr error } // VarAssignment represents a variable assignment in Set, such as set global a = 1. diff --git a/expression/scalar_function.go b/expression/scalar_function.go index dd7805a6c282f..220eeb090a5b1 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -30,15 +30,9 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/hack" ) -// error definitions. -var ( - ErrNoDB = dbterror.ClassOptimizer.NewStd(mysql.ErrNoDB) -) - // ScalarFunction is the function that returns a value. type ScalarFunction struct { FuncName model.CIStr @@ -286,7 +280,7 @@ func (sf *ScalarFunction) Clone() Expression { Function: sf.Function.Clone(), hashcode: sf.hashcode, } - c.SetCharsetAndCollation(sf.CharsetAndCollation(sf.GetCtx())) + c.SetCharsetAndCollation(sf.CharsetAndCollation()) c.SetCoercibility(sf.Coercibility()) return c } @@ -558,12 +552,12 @@ func (sf *ScalarFunction) SetCoercibility(val Coercibility) { sf.Function.SetCoercibility(val) } -// CharsetAndCollation ... -func (sf *ScalarFunction) CharsetAndCollation(ctx sessionctx.Context) (string, string) { - return sf.Function.CharsetAndCollation(ctx) +// CharsetAndCollation gets charset and collation. +func (sf *ScalarFunction) CharsetAndCollation() (string, string) { + return sf.Function.CharsetAndCollation() } -// SetCharsetAndCollation ... +// SetCharsetAndCollation sets charset and collation. func (sf *ScalarFunction) SetCharsetAndCollation(chs, coll string) { sf.Function.SetCharsetAndCollation(chs, coll) } diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 13f5d81380e7e..f2e94aa7c093e 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -15,9 +15,10 @@ package expression_test import ( + "fmt" "math" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" @@ -25,44 +26,26 @@ import ( "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/printer" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" "golang.org/x/net/context" ) -var _ = SerialSuites(&testInferTypeSuite{}) - -type typeInferTestCase struct { - sql string - tp byte - chs string - flag uint - flen int - decimal int -} - -type testInferTypeSuite struct { - *parser.Parser -} - -func (s *testInferTypeSuite) SetUpSuite(c *C) { - s.Parser = parser.New() -} - -func (s *testInferTypeSuite) TearDownSuite(c *C) { -} - -func (s *testInferTypeSuite) TestInferType(c *C) { +func TestInferType(t *testing.T) { + t.Parallel() store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { dom.Close() - store.Close() + err = store.Close() + require.NoError(t, err) }() + s := InferTypeSuite{} se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) + require.NoError(t, err) + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") sql := `create table t ( @@ -131,36 +114,47 @@ func (s *testInferTypeSuite) TestInferType(c *C) { tests = append(tests, s.createTestCase4MiscellaneousFunc()...) tests = append(tests, s.createTestCase4GetVarFunc()...) - sctx := testKit.Se.(sessionctx.Context) - c.Assert(sctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, mysql.DefaultCharset), IsNil) - c.Assert(sctx.GetSessionVars().SetSystemVar(variable.CollationConnection, mysql.DefaultCollationName), IsNil) + sctx := testKit.Session().(sessionctx.Context) + require.NoError(t, sctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, mysql.DefaultCharset)) + require.NoError(t, sctx.GetSessionVars().SetSystemVar(variable.CollationConnection, mysql.DefaultCollationName)) ctx := context.Background() + par := parser.New() for _, tt := range tests { sql := "select " + tt.sql + " from t" - comment := Commentf("for %s", sql) - stmt, err := s.ParseOneStmt(sql, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("for %s", sql) + stmt, err := par.ParseOneStmt(sql, "", "") + require.NoError(t, err, comment) err = se.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmt, plannercore.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil, comment) + require.NoError(t, err, comment) p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil, comment) + require.NoError(t, err, comment) tp := p.Schema().Columns[0].RetType - - c.Check(tp.Tp, Equals, tt.tp, comment) - c.Check(tp.Charset, Equals, tt.chs, comment) - c.Check(tp.Flag, Equals, tt.flag, comment) - c.Check(tp.Flen, Equals, tt.flen, comment) - c.Check(tp.Decimal, Equals, tt.decimal, comment) + require.Equal(t, tt.tp, tp.Tp, comment) + require.Equal(t, tt.chs, tp.Charset, comment) + require.Equal(t, tt.flag, tp.Flag, comment) + require.Equal(t, tt.flen, tp.Flen, comment) + require.Equal(t, tt.decimal, tp.Decimal, comment) } } -func (s *testInferTypeSuite) createTestCase4Constants() []typeInferTestCase { +type typeInferTestCase struct { + sql string + tp byte + chs string + flag uint + flen int + decimal int +} + +type InferTypeSuite struct{} + +func (s *InferTypeSuite) createTestCase4Constants() []typeInferTestCase { return []typeInferTestCase{ {"1", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 1, 0}, {"-1", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 2, 0}, @@ -185,7 +179,7 @@ func (s *testInferTypeSuite) createTestCase4Constants() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4Cast() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4Cast() []typeInferTestCase { return []typeInferTestCase{ {"CAST(c_int_d AS BINARY)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, -1, -1}, // TODO: Flen should be 11. {"CAST(c_int_d AS BINARY(5))", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 5, -1}, @@ -228,7 +222,7 @@ func (s *testInferTypeSuite) createTestCase4Cast() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4Columns() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4Columns() []typeInferTestCase { return []typeInferTestCase{ {"c_bit ", mysql.TypeBit, charset.CharsetBin, mysql.UnsignedFlag, 10, 0}, {"c_year ", mysql.TypeYear, charset.CharsetBin, mysql.UnsignedFlag | mysql.ZerofillFlag, 4, 0}, @@ -264,7 +258,7 @@ func (s *testInferTypeSuite) createTestCase4Columns() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { return []typeInferTestCase{ {"strcmp(c_char, c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 2, 0}, {"space(c_int_d)", mysql.TypeLongBlob, mysql.DefaultCharset, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, @@ -513,7 +507,7 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4MathFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4MathFuncs() []typeInferTestCase { return []typeInferTestCase{ {"cos(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"sin(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, @@ -721,7 +715,7 @@ func (s *testInferTypeSuite) createTestCase4MathFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4ArithmeticFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4ArithmeticFuncs() []typeInferTestCase { return []typeInferTestCase{ {"c_int_d + c_int_d", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, {"c_int_d + c_bigint_d", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, @@ -815,7 +809,7 @@ func (s *testInferTypeSuite) createTestCase4ArithmeticFuncs() []typeInferTestCas } } -func (s *testInferTypeSuite) createTestCase4LogicalFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4LogicalFuncs() []typeInferTestCase { return []typeInferTestCase{ {"c_int_d and c_int_d", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, {"c_int_d xor c_int_d", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, @@ -825,7 +819,7 @@ func (s *testInferTypeSuite) createTestCase4LogicalFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase { return []typeInferTestCase{ {"ifnull(c_int_d, c_int_d)", mysql.TypeLong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, {"ifnull(c_int_d, c_decimal)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 14, 3}, @@ -853,7 +847,7 @@ func (s *testInferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4Aggregations() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4Aggregations() []typeInferTestCase { return []typeInferTestCase{ {"sum(c_int_d)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 32, 0}, {"sum(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, @@ -876,7 +870,7 @@ func (s *testInferTypeSuite) createTestCase4Aggregations() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4InfoFunc() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4InfoFunc() []typeInferTestCase { return []typeInferTestCase{ {"last_insert_id( )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag | mysql.NotNullFlag, mysql.MaxIntWidth, 0}, {"last_insert_id(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag, mysql.MaxIntWidth, 0}, @@ -890,7 +884,7 @@ func (s *testInferTypeSuite) createTestCase4InfoFunc() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4EncryptionFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4EncryptionFuncs() []typeInferTestCase { return []typeInferTestCase{ {"md5(c_int_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 32, types.UnspecifiedLength}, {"md5(c_bigint_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 32, types.UnspecifiedLength}, @@ -1022,7 +1016,7 @@ func (s *testInferTypeSuite) createTestCase4EncryptionFuncs() []typeInferTestCas } } -func (s *testInferTypeSuite) createTestCase4CompareFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4CompareFuncs() []typeInferTestCase { return []typeInferTestCase{ {"coalesce(c_int_d, 1)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, {"coalesce(NULL, c_int_d)", mysql.TypeLong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, @@ -1066,7 +1060,7 @@ func (s *testInferTypeSuite) createTestCase4CompareFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4Miscellaneous() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4Miscellaneous() []typeInferTestCase { return []typeInferTestCase{ {"sleep(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 21, 0}, {"sleep(c_float_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 21, 0}, @@ -1169,7 +1163,7 @@ func (s *testInferTypeSuite) createTestCase4Miscellaneous() []typeInferTestCase } } -func (s *testInferTypeSuite) createTestCase4OpFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4OpFuncs() []typeInferTestCase { return []typeInferTestCase{ {"c_int_d is true", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, {"c_decimal is true", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, @@ -1195,7 +1189,7 @@ func (s *testInferTypeSuite) createTestCase4OpFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4OtherFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4OtherFuncs() []typeInferTestCase { return []typeInferTestCase{ {"1 in (c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, {"1 in (c_decimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, @@ -1227,7 +1221,7 @@ func (s *testInferTypeSuite) createTestCase4OtherFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { return []typeInferTestCase{ {`time_format('150:02:28', '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8MB4, mysql.NotNullFlag, 44, types.UnspecifiedLength}, {`time_format(123456, '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8MB4, mysql.NotNullFlag, 44, types.UnspecifiedLength}, @@ -1908,7 +1902,7 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4LikeFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4LikeFuncs() []typeInferTestCase { return []typeInferTestCase{ {"c_int_d rlike c_text_d", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, {"c_bigint_d rlike c_text_d", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.IsBooleanFlag, 1, 0}, @@ -1946,7 +1940,7 @@ func (s *testInferTypeSuite) createTestCase4LikeFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4Literals() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4Literals() []typeInferTestCase { return []typeInferTestCase{ {"time '00:00:00'", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 10, 0}, {"time '00'", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 10, 0}, @@ -1958,7 +1952,7 @@ func (s *testInferTypeSuite) createTestCase4Literals() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4JSONFuncs() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4JSONFuncs() []typeInferTestCase { return []typeInferTestCase{ {"json_type(c_json)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 51, types.UnspecifiedLength}, // TODO: Flen of json_unquote doesn't follow MySQL now. @@ -1974,7 +1968,7 @@ func (s *testInferTypeSuite) createTestCase4JSONFuncs() []typeInferTestCase { } } -func (s *testInferTypeSuite) createTestCase4MiscellaneousFunc() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4MiscellaneousFunc() []typeInferTestCase { return []typeInferTestCase{ {"get_lock(c_char, c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, {"get_lock(c_char, c_bigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, @@ -1994,7 +1988,7 @@ func (s *testInferTypeSuite) createTestCase4MiscellaneousFunc() []typeInferTestC } } -func (s *testInferTypeSuite) createTestCase4GetVarFunc() []typeInferTestCase { +func (s *InferTypeSuite) createTestCase4GetVarFunc() []typeInferTestCase { return []typeInferTestCase{ {"@a", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"@b", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, diff --git a/expression/util_test.go b/expression/util_test.go index 5d44c8137eee9..75e6b9c67113f 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -608,7 +608,7 @@ func (m *MockExpr) SetCoercibility(Coercibility) func (m *MockExpr) Repertoire() Repertoire { return UNICODE } func (m *MockExpr) SetRepertoire(Repertoire) {} -func (m *MockExpr) CharsetAndCollation(ctx sessionctx.Context) (string, string) { +func (m *MockExpr) CharsetAndCollation() (string, string) { return "", "" } func (m *MockExpr) SetCharsetAndCollation(chs, coll string) {} diff --git a/go.mod b/go.mod index e9d31236a7222..aa4dc6c700ff1 100644 --- a/go.mod +++ b/go.mod @@ -48,7 +48,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20211009033009-93128226aaa3 github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20211011060348-d957056f1551 + github.com/pingcap/kvproto v0.0.0-20211029081837-3c7bd947cf9b github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5 github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible @@ -65,8 +65,8 @@ require ( 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-alpha.0.20211102120533-b8cc5a319d96 - github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae + github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211115071040-a3f1c41ac1a0 + github.com/tikv/pd v1.1.0-beta.0.20211104095303-69c86d05d379 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 diff --git a/go.sum b/go.sum index 59598006c823b..1237e7f202a53 100644 --- a/go.sum +++ b/go.sum @@ -585,8 +585,8 @@ github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO 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-20210915062418-0f5764a128ad/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211011060348-d957056f1551 h1:aRx2l2TAeYNPPUc+lk5dEFCXfUGxR/C2fbt/YA5nqiQ= -github.com/pingcap/kvproto v0.0.0-20211011060348-d957056f1551/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20211029081837-3c7bd947cf9b h1:/aj6ITlHSJZmsm4hIMOgJAAZti+Dmq11tCyKedA6Dcs= +github.com/pingcap/kvproto v0.0.0-20211029081837-3c7bd947cf9b/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= @@ -597,6 +597,7 @@ github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041 github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5 h1:7rvAtZe/ZUzOKzgriNPQoBNvleJXBk4z7L3Z47+tS98= github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5/go.mod h1:XsOaV712rUk63aOEKYP9PhXTIE3FMNHmC2r1wX5wElY= github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= +github.com/pingcap/tidb-dashboard v0.0.0-20211031170437-08e58c069a2a/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-20211105090418-71142a4d40e3 h1:xnp/Qkk5gELlB8TaY6oro0JNXMBXTafNVxU/vbrNU8I= @@ -712,10 +713,11 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK 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-alpha.0.20211102120533-b8cc5a319d96 h1:+PffHhOZm7OpC+VrL10ahT/KwloECT4ZtSJ0D8SzmYQ= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211102120533-b8cc5a319d96/go.mod h1:iiwtsCxcbNLK5i9VRYGvdcihgHXTKy2ukWjoaJsrphg= -github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae h1:PmnkhiOopgMZYDQ7Htj1kt/zwW4MEOUL+Dem6WLZISY= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211115071040-a3f1c41ac1a0 h1:c12Pv8Xks4oubDr/uHHxrlBkwGJFqKZUEIUemHV794g= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211115071040-a3f1c41ac1a0/go.mod h1:iiwtsCxcbNLK5i9VRYGvdcihgHXTKy2ukWjoaJsrphg= 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.20211104095303-69c86d05d379 h1:nFm1jQDz1iRktoyV2SyM5zVk6+PJHQNunJZ7ZJcqzAo= +github.com/tikv/pd v1.1.0-beta.0.20211104095303-69c86d05d379/go.mod h1:y+09hAUXJbrd4c0nktL74zXDDuD7atGtfOKxL90PCOE= github.com/tklauser/go-sysconf v0.3.4 h1:HT8SVixZd3IzLdfs/xlpq0jeSfTX57g1v6wB1EuzV7M= github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= github.com/tklauser/numcpus v0.2.1 h1:ct88eFm+Q7m2ZfXJdan1xYoXKlmwsfP+k88q05KvlZc= diff --git a/infoschema/perfschema/tables_serial_test.go b/infoschema/perfschema/tables_serial_test.go new file mode 100644 index 0000000000000..f8126a57b2b7c --- /dev/null +++ b/infoschema/perfschema/tables_serial_test.go @@ -0,0 +1,159 @@ +// 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 perfschema_test + +import ( + "fmt" + "io" + "net/http" + "net/http/httptest" + "os" + "path/filepath" + "runtime/pprof" + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestTiKVProfileCPU(t *testing.T) { + store, clean := newMockStore(t) + defer clean() + + router := http.NewServeMux() + mockServer := httptest.NewServer(router) + mockAddr := strings.TrimPrefix(mockServer.URL, "http://") + defer mockServer.Close() + + // mock tikv profile + copyHandler := func(filename string) http.HandlerFunc { + return func(w http.ResponseWriter, _ *http.Request) { + file, err := os.Open(filepath.Join(currentSourceDir(), filename)) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + defer func() { terror.Log(file.Close()) }() + _, err = io.Copy(w, file) + terror.Log(err) + } + } + router.HandleFunc("/debug/pprof/profile", copyHandler("testdata/tikv.cpu.profile")) + + // failpoint setting + servers := []string{ + strings.Join([]string{"tikv", mockAddr, mockAddr}, ","), + strings.Join([]string{"pd", mockAddr, mockAddr}, ","), + } + fpExpr := strings.Join(servers, ";") + fpName := "github.com/pingcap/tidb/infoschema/perfschema/mockRemoteNodeStatusAddress" + require.NoError(t, failpoint.Enable(fpName, fmt.Sprintf(`return("%s")`, fpExpr))) + defer func() { require.NoError(t, failpoint.Disable(fpName)) }() + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use performance_schema") + result := tk.MustQuery("select function, percent_abs, percent_rel from tikv_profile_cpu where depth < 3") + + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + result.Check(testkit.Rows( + "root 100% 100%", + "├─tikv::server::load_statistics::linux::ThreadLoadStatistics::record::h59facb8d680e7794 75.00% 75.00%", + "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 75.00% 100%", + "├─nom::nom::digit::h905aaaeff7d8ec8e 16.07% 16.07%", + "│ ├─ as core::iter::traits::iterator::Iterator>::next::h16936f9061bb75e4 6.25% 38.89%", + "│ ├─Unknown 3.57% 22.22%", + "│ ├─<&u8 as nom::traits::AsChar>::is_dec_digit::he9eacc3fad26ab81 2.68% 16.67%", + "│ ├─<&[u8] as nom::traits::InputIter>::iter_indices::h6192338433683bff 1.79% 11.11%", + "│ └─<&[T] as nom::traits::Slice>>::slice::h38d31f11f84aa302 1.79% 11.11%", + "├─::realloc::h5199c50710ab6f9d 1.79% 1.79%", + "│ └─rallocx 1.79% 100%", + "├─::dealloc::hea83459aa98dd2dc 1.79% 1.79%", + "│ └─sdallocx 1.79% 100%", + "├─::alloc::hc7962e02169a5c56 0.89% 0.89%", + "│ └─mallocx 0.89% 100%", + "├─engine::rocks::util::engine_metrics::flush_engine_iostall_properties::h64a7661c95aa1db7 0.89% 0.89%", + "│ └─rocksdb::rocksdb::DB::get_map_property_cf::h9722f9040411af44 0.89% 100%", + "├─core::ptr::real_drop_in_place::h8def0d99e7136f33 0.89% 0.89%", + "│ └─ as core::ops::drop::Drop>::drop::h9b59b303bffde02c 0.89% 100%", + "├─tikv_util::metrics::threads_linux::ThreadInfoStatistics::record::ha8cc290b3f46af88 0.89% 0.89%", + "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 0.89% 100%", + "├─crossbeam_utils::backoff::Backoff::snooze::h5c121ef4ce616a3c 0.89% 0.89%", + "│ └─core::iter::range::>::next::hdb23ceb766e7a91f 0.89% 100%", + "└─::next::he129c78b3deb639d 0.89% 0.89%", + " └─Unknown 0.89% 100%")) + + // We can use current processe profile to mock profile of PD because the PD has the + // same way of retrieving profile with TiDB. And the purpose of this test case is used + // to make sure all profile HTTP API have been accessed. + accessed := map[string]struct{}{} + handlerFactory := func(name string, debug ...int) func(w http.ResponseWriter, _ *http.Request) { + debugLevel := 0 + if len(debug) > 0 { + debugLevel = debug[0] + } + return func(w http.ResponseWriter, _ *http.Request) { + profile := pprof.Lookup(name) + if profile == nil { + http.Error(w, fmt.Sprintf("profile %s not found", name), http.StatusBadRequest) + return + } + if err := profile.WriteTo(w, debugLevel); err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + accessed[name] = struct{}{} + } + } + + // mock PD profile + router.HandleFunc("/pd/api/v1/debug/pprof/profile", copyHandler("../../util/profile/testdata/test.pprof")) + router.HandleFunc("/pd/api/v1/debug/pprof/heap", handlerFactory("heap")) + router.HandleFunc("/pd/api/v1/debug/pprof/mutex", handlerFactory("mutex")) + router.HandleFunc("/pd/api/v1/debug/pprof/allocs", handlerFactory("allocs")) + router.HandleFunc("/pd/api/v1/debug/pprof/block", handlerFactory("block")) + router.HandleFunc("/pd/api/v1/debug/pprof/goroutine", handlerFactory("goroutine", 2)) + + tk.MustQuery("select * from pd_profile_cpu where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_memory where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_mutex where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_allocs where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_block where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_goroutines") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + require.Lenf(t, accessed, 5, "expect all HTTP API had been accessed, but found: %v", accessed) +} diff --git a/infoschema/perfschema/tables_test.go b/infoschema/perfschema/tables_test.go index 981243e663260..f5e910635d5fa 100644 --- a/infoschema/perfschema/tables_test.go +++ b/infoschema/perfschema/tables_test.go @@ -15,21 +15,12 @@ package perfschema_test import ( - "fmt" - "io" - "net/http" - "net/http/httptest" - "os" "path/filepath" "runtime" - "runtime/pprof" - "strings" "testing" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/infoschema/perfschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" @@ -56,135 +47,6 @@ func TestPerfSchemaTables(t *testing.T) { tk.MustQuery("select * from events_stages_history_long").Check(testkit.Rows()) } -func TestTiKVProfileCPU(t *testing.T) { - t.Parallel() - - store, clean := newMockStore(t) - defer clean() - - router := http.NewServeMux() - mockServer := httptest.NewServer(router) - mockAddr := strings.TrimPrefix(mockServer.URL, "http://") - defer mockServer.Close() - - // mock tikv profile - copyHandler := func(filename string) http.HandlerFunc { - return func(w http.ResponseWriter, _ *http.Request) { - file, err := os.Open(filepath.Join(currentSourceDir(), filename)) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - defer func() { terror.Log(file.Close()) }() - _, err = io.Copy(w, file) - terror.Log(err) - } - } - router.HandleFunc("/debug/pprof/profile", copyHandler("testdata/tikv.cpu.profile")) - - // failpoint setting - servers := []string{ - strings.Join([]string{"tikv", mockAddr, mockAddr}, ","), - strings.Join([]string{"pd", mockAddr, mockAddr}, ","), - } - fpExpr := strings.Join(servers, ";") - fpName := "github.com/pingcap/tidb/infoschema/perfschema/mockRemoteNodeStatusAddress" - require.NoError(t, failpoint.Enable(fpName, fmt.Sprintf(`return("%s")`, fpExpr))) - defer func() { require.NoError(t, failpoint.Disable(fpName)) }() - - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use performance_schema") - result := tk.MustQuery("select function, percent_abs, percent_rel from tikv_profile_cpu where depth < 3") - - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - result.Check(testkit.Rows( - "root 100% 100%", - "├─tikv::server::load_statistics::linux::ThreadLoadStatistics::record::h59facb8d680e7794 75.00% 75.00%", - "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 75.00% 100%", - "├─nom::nom::digit::h905aaaeff7d8ec8e 16.07% 16.07%", - "│ ├─ as core::iter::traits::iterator::Iterator>::next::h16936f9061bb75e4 6.25% 38.89%", - "│ ├─Unknown 3.57% 22.22%", - "│ ├─<&u8 as nom::traits::AsChar>::is_dec_digit::he9eacc3fad26ab81 2.68% 16.67%", - "│ ├─<&[u8] as nom::traits::InputIter>::iter_indices::h6192338433683bff 1.79% 11.11%", - "│ └─<&[T] as nom::traits::Slice>>::slice::h38d31f11f84aa302 1.79% 11.11%", - "├─::realloc::h5199c50710ab6f9d 1.79% 1.79%", - "│ └─rallocx 1.79% 100%", - "├─::dealloc::hea83459aa98dd2dc 1.79% 1.79%", - "│ └─sdallocx 1.79% 100%", - "├─::alloc::hc7962e02169a5c56 0.89% 0.89%", - "│ └─mallocx 0.89% 100%", - "├─engine::rocks::util::engine_metrics::flush_engine_iostall_properties::h64a7661c95aa1db7 0.89% 0.89%", - "│ └─rocksdb::rocksdb::DB::get_map_property_cf::h9722f9040411af44 0.89% 100%", - "├─core::ptr::real_drop_in_place::h8def0d99e7136f33 0.89% 0.89%", - "│ └─ as core::ops::drop::Drop>::drop::h9b59b303bffde02c 0.89% 100%", - "├─tikv_util::metrics::threads_linux::ThreadInfoStatistics::record::ha8cc290b3f46af88 0.89% 0.89%", - "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 0.89% 100%", - "├─crossbeam_utils::backoff::Backoff::snooze::h5c121ef4ce616a3c 0.89% 0.89%", - "│ └─core::iter::range::>::next::hdb23ceb766e7a91f 0.89% 100%", - "└─::next::he129c78b3deb639d 0.89% 0.89%", - " └─Unknown 0.89% 100%")) - - // We can use current processe profile to mock profile of PD because the PD has the - // same way of retrieving profile with TiDB. And the purpose of this test case is used - // to make sure all profile HTTP API have been accessed. - accessed := map[string]struct{}{} - handlerFactory := func(name string, debug ...int) func(w http.ResponseWriter, _ *http.Request) { - debugLevel := 0 - if len(debug) > 0 { - debugLevel = debug[0] - } - return func(w http.ResponseWriter, _ *http.Request) { - profile := pprof.Lookup(name) - if profile == nil { - http.Error(w, fmt.Sprintf("profile %s not found", name), http.StatusBadRequest) - return - } - if err := profile.WriteTo(w, debugLevel); err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - accessed[name] = struct{}{} - } - } - - // mock PD profile - router.HandleFunc("/pd/api/v1/debug/pprof/profile", copyHandler("../../util/profile/testdata/test.pprof")) - router.HandleFunc("/pd/api/v1/debug/pprof/heap", handlerFactory("heap")) - router.HandleFunc("/pd/api/v1/debug/pprof/mutex", handlerFactory("mutex")) - router.HandleFunc("/pd/api/v1/debug/pprof/allocs", handlerFactory("allocs")) - router.HandleFunc("/pd/api/v1/debug/pprof/block", handlerFactory("block")) - router.HandleFunc("/pd/api/v1/debug/pprof/goroutine", handlerFactory("goroutine", 2)) - - tk.MustQuery("select * from pd_profile_cpu where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_memory where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_mutex where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_allocs where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_block where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_goroutines") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - require.Lenf(t, accessed, 5, "expect all HTTP API had been accessed, but found: %v", accessed) -} - func newMockStore(t *testing.T) (store kv.Storage, clean func()) { var err error store, err = mockstore.NewMockStore() diff --git a/infoschema/tables.go b/infoschema/tables.go index 5fba2a2b22b86..78ee3c218bfa5 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -387,7 +387,7 @@ var tablesCols = []columnInfo{ {name: "TIDB_ROW_ID_SHARDING_INFO", tp: mysql.TypeVarchar, size: 255}, {name: "TIDB_PK_TYPE", tp: mysql.TypeVarchar, size: 64}, {name: "TIDB_PLACEMENT_POLICY_NAME", tp: mysql.TypeVarchar, size: 64}, - {name: "TIDB_DIRECT_PLACEMENT", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, + {name: "TIDB_DIRECT_PLACEMENT", tp: mysql.TypeVarchar, size: 1024}, } // See: http://dev.mysql.com/doc/refman/5.7/en/columns-table.html @@ -560,7 +560,7 @@ var partitionsCols = []columnInfo{ {name: "TABLESPACE_NAME", tp: mysql.TypeVarchar, size: 64}, {name: "TIDB_PARTITION_ID", tp: mysql.TypeLonglong, size: 21}, {name: "TIDB_PLACEMENT_POLICY_NAME", tp: mysql.TypeVarchar, size: 64}, - {name: "TIDB_DIRECT_PLACEMENT", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, + {name: "TIDB_DIRECT_PLACEMENT", tp: mysql.TypeVarchar, size: 1024}, } var tableConstraintsCols = []columnInfo{ @@ -1451,16 +1451,16 @@ var tableAttributesCols = []columnInfo{ var tablePlacementRulesCols = []columnInfo{ {name: "POLICY_ID", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, {name: "CATALOG_NAME", tp: mysql.TypeVarchar, size: 512, flag: mysql.NotNullFlag}, - {name: "POLICY_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // Catalog wide policy - {name: "SCHEMA_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // System policy does not have a schema - {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // Schema level rules does not have a table - {name: "PARTITION_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // Table level rules does not have a partition - {name: "PRIMARY_REGION", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, - {name: "REGIONS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, - {name: "CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, - {name: "LEADER_CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, - {name: "FOLLOWER_CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, - {name: "LEARNER_CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, + {name: "POLICY_NAME", tp: mysql.TypeVarchar, size: 64}, // Catalog wide policy + {name: "SCHEMA_NAME", tp: mysql.TypeVarchar, size: 64}, // System policy does not have a schema + {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, // Schema level rules does not have a table + {name: "PARTITION_NAME", tp: mysql.TypeVarchar, size: 64}, // Table level rules does not have a partition + {name: "PRIMARY_REGION", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "REGIONS", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "CONSTRAINTS", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "LEADER_CONSTRAINTS", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "FOLLOWER_CONSTRAINTS", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "LEARNER_CONSTRAINTS", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, {name: "SCHEDULE", tp: mysql.TypeVarchar, size: 20, flag: mysql.NotNullFlag}, // EVEN or MAJORITY_IN_PRIMARY {name: "FOLLOWERS", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, {name: "LEARNERS", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, diff --git a/kv/fault_injection.go b/kv/fault_injection.go index af875d39ea1b6..0cd32dca1b5ac 100644 --- a/kv/fault_injection.go +++ b/kv/fault_injection.go @@ -58,17 +58,8 @@ func NewInjectedStore(store Storage, cfg *InjectionConfig) Storage { } // Begin creates an injected Transaction. -func (s *InjectedStore) Begin() (Transaction, error) { - txn, err := s.Storage.Begin() - return &InjectedTransaction{ - Transaction: txn, - cfg: s.cfg, - }, err -} - -// BeginWithOption creates an injected Transaction with given option. -func (s *InjectedStore) BeginWithOption(option tikv.StartTSOption) (Transaction, error) { - txn, err := s.Storage.BeginWithOption(option) +func (s *InjectedStore) Begin(opts ...tikv.TxnOption) (Transaction, error) { + txn, err := s.Storage.Begin(opts...) return &InjectedTransaction{ Transaction: txn, cfg: s.cfg, diff --git a/kv/fault_injection_test.go b/kv/fault_injection_test.go index 743d453b5dd68..754716d4bcdfb 100644 --- a/kv/fault_injection_test.go +++ b/kv/fault_injection_test.go @@ -37,7 +37,7 @@ func TestFaultInjectionBasic(t *testing.T) { txn, err := storage.Begin() require.Nil(t, err) - _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(GlobalTxnScope).SetStartTS(0)) + _, err = storage.Begin(tikv.WithTxnScope(GlobalTxnScope), tikv.WithStartTS(0)) require.Nil(t, err) ver := Version{Ver: 1} diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 166acf202794b..2542e6f7e5b55 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -63,6 +63,7 @@ func (t *mockTxn) IsReadOnly() bool { func (t *mockTxn) StartTS() uint64 { return uint64(0) } + func (t *mockTxn) Get(ctx context.Context, k Key) ([]byte, error) { return nil, nil } @@ -82,6 +83,7 @@ func (t *mockTxn) IterReverse(k Key) (Iterator, error) { func (t *mockTxn) Set(k Key, v []byte) error { return nil } + func (t *mockTxn) Delete(k Key) error { return nil } @@ -115,7 +117,6 @@ func (t *mockTxn) Flush() (int, error) { } func (t *mockTxn) Discard() { - } func (t *mockTxn) Reset() { @@ -123,7 +124,6 @@ func (t *mockTxn) Reset() { } func (t *mockTxn) SetVars(vars interface{}) { - } func (t *mockTxn) GetVars() interface{} { @@ -131,7 +131,6 @@ func (t *mockTxn) GetVars() interface{} { } func (t *mockTxn) CacheTableInfo(id int64, info *model.TableInfo) { - } func (t *mockTxn) GetTableInfo(id int64) *model.TableInfo { @@ -139,11 +138,11 @@ func (t *mockTxn) GetTableInfo(id int64) *model.TableInfo { } func (t *mockTxn) SetDiskFullOpt(level kvrpcpb.DiskFullOpt) { - //TODO nothing + // TODO nothing } func (t *mockTxn) ClearDiskFullOpt() { - //TODO nothing + // TODO nothing } // newMockTxn new a mockTxn. @@ -155,14 +154,9 @@ func newMockTxn() Transaction { } // mockStorage is used to start a must commit-failed txn. -type mockStorage struct { -} +type mockStorage struct{} -func (s *mockStorage) Begin() (Transaction, error) { - return newMockTxn(), nil -} - -func (s *mockStorage) BeginWithOption(option tikv.StartTSOption) (Transaction, error) { +func (s *mockStorage) Begin(opts ...tikv.TxnOption) (Transaction, error) { return newMockTxn(), nil } @@ -243,7 +237,6 @@ func (s *mockSnapshot) Get(ctx context.Context, k Key) ([]byte, error) { } func (s *mockSnapshot) SetPriority(priority int) { - } func (s *mockSnapshot) BatchGet(ctx context.Context, keys []Key) (map[string][]byte, error) { diff --git a/kv/kv.go b/kv/kv.go index ca3dbd1d239a2..a6ce83fd8caf0 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -405,9 +405,7 @@ type Driver interface { // Isolation should be at least SI(SNAPSHOT ISOLATION) type Storage interface { // Begin a global transaction - Begin() (Transaction, error) - // BeginWithOption begins a transaction with given option - BeginWithOption(option tikv.StartTSOption) (Transaction, error) + Begin(opts ...tikv.TxnOption) (Transaction, error) // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. // if ver is MaxVersion or > current max committed version, we will use current version for this snapshot. GetSnapshot(ver Version) Snapshot diff --git a/parser/ast/stats.go b/parser/ast/stats.go index b71cf8b8fa592..55bb1f7abbdd8 100644 --- a/parser/ast/stats.go +++ b/parser/ast/stats.go @@ -40,7 +40,7 @@ type AnalyzeTableStmt struct { // HistogramOperation is set in "ANALYZE TABLE ... UPDATE/DROP HISTOGRAM ..." statement. HistogramOperation HistogramOperationType // ColumnNames indicate the columns whose statistics need to be collected. - ColumnNames []*ColumnName + ColumnNames []model.CIStr ColumnChoice model.ColumnChoice } @@ -129,7 +129,7 @@ func (n *AnalyzeTableStmt) Restore(ctx *format.RestoreCtx) error { if i != 0 { ctx.WritePlain(",") } - ctx.WriteName(columnName.Name.O) + ctx.WriteName(columnName.O) } } } @@ -144,7 +144,7 @@ func (n *AnalyzeTableStmt) Restore(ctx *format.RestoreCtx) error { if i != 0 { ctx.WritePlain(",") } - ctx.WriteName(columnName.Name.O) + ctx.WriteName(columnName.O) } } if n.IndexFlag { diff --git a/parser/charset/charset.go b/parser/charset/charset.go index a0b665ecf2b4c..e47a65d4c89df 100644 --- a/parser/charset/charset.go +++ b/parser/charset/charset.go @@ -148,7 +148,13 @@ func GetCharsetInfoByID(coID int) (string, string, error) { if collation, ok := collationsIDMap[coID]; ok { return collation.CharsetName, collation.Name, nil } - return "", "", errors.Errorf("Unknown charset id %d", coID) + + // TODO: uncomment it when issue #29697 be closed + // log.Warn( + // "Unable to get collation name from collation ID, return default charset and collation instead.", + // zap.Int("ID", coID), + // zap.Stack("stack")) + return mysql.DefaultCharset, mysql.DefaultCollationName, errors.Errorf("Unknown collation id %d", coID) } // GetCollations returns a list for all collations. diff --git a/parser/model/ddl.go b/parser/model/ddl.go index acc430c20d14f..622b2c460730e 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -93,6 +93,7 @@ const ( ActionAlterTablePlacement ActionType = 56 ActionAlterCacheTable ActionType = 57 ActionAlterTableStatsOptions ActionType = 58 + ActionAlterNoCacheTable ActionType = 59 ) var actionMap = map[ActionType]string{ diff --git a/parser/parser.go b/parser/parser.go index 307fa1ff4fcb6..fd74b978614f2 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -1279,7 +1279,7 @@ var ( 57903: 457, // weightString (1422x) 57488: 458, // on (1368x) 40: 459, // '(' (1283x) - 57568: 460, // with (1186x) + 57568: 460, // with (1183x) 57349: 461, // stringLit (1168x) 58074: 462, // not2 (1155x) 57481: 463, // not (1100x) @@ -1548,7 +1548,7 @@ var ( 57549: 726, // unsigned (47x) 57495: 727, // over (45x) 57571: 728, // zerofill (45x) - 58174: 729, // ColumnName (44x) + 58174: 729, // ColumnName (40x) 58363: 730, // LengthNum (40x) 57400: 731, // deleteKwd (38x) 57404: 732, // distinct (36x) @@ -1590,30 +1590,30 @@ var ( 58232: 768, // DistinctKwd (15x) 58322: 769, // IfNotExists (15x) 58417: 770, // OptFieldLen (15x) - 58175: 771, // ColumnNameList (14x) - 58233: 772, // DistinctOpt (14x) - 57411: 773, // enclosed (14x) - 58349: 774, // InsertIntoStmt (14x) - 58448: 775, // PartitionNameList (14x) - 58487: 776, // ReplaceIntoStmt (14x) - 58628: 777, // UpdateStmt (14x) - 58659: 778, // WhereClause (14x) - 58660: 779, // WhereClauseOptional (14x) - 58225: 780, // DefaultKwdOpt (13x) - 57412: 781, // escaped (13x) - 57491: 782, // optionally (13x) - 58595: 783, // TableNameList (13x) - 58357: 784, // JoinTable (12x) - 58411: 785, // OptBinary (12x) - 58503: 786, // RolenameComposed (12x) - 58591: 787, // TableFactor (12x) - 58604: 788, // TableRef (12x) - 58124: 789, // AnalyzeOptionListOpt (11x) - 58229: 790, // DeleteWithUsingStmt (11x) - 58263: 791, // ExprOrDefault (11x) - 58292: 792, // FromOrIn (11x) - 58618: 793, // TimestampUnit (11x) - 58163: 794, // CharsetName (10x) + 58233: 771, // DistinctOpt (14x) + 57411: 772, // enclosed (14x) + 58349: 773, // InsertIntoStmt (14x) + 58448: 774, // PartitionNameList (14x) + 58487: 775, // ReplaceIntoStmt (14x) + 58628: 776, // UpdateStmt (14x) + 58659: 777, // WhereClause (14x) + 58660: 778, // WhereClauseOptional (14x) + 58225: 779, // DefaultKwdOpt (13x) + 57412: 780, // escaped (13x) + 57491: 781, // optionally (13x) + 58595: 782, // TableNameList (13x) + 58357: 783, // JoinTable (12x) + 58411: 784, // OptBinary (12x) + 58503: 785, // RolenameComposed (12x) + 58591: 786, // TableFactor (12x) + 58604: 787, // TableRef (12x) + 58124: 788, // AnalyzeOptionListOpt (11x) + 58229: 789, // DeleteWithUsingStmt (11x) + 58263: 790, // ExprOrDefault (11x) + 58292: 791, // FromOrIn (11x) + 58618: 792, // TimestampUnit (11x) + 58163: 793, // CharsetName (10x) + 58175: 794, // ColumnNameList (10x) 58228: 795, // DeleteFromStmt (10x) 58396: 796, // NotSym (10x) 58438: 797, // OrderByOptional (10x) @@ -1675,350 +1675,350 @@ var ( 58172: 853, // ColumnKeywordOpt (5x) 58277: 854, // FieldOpt (5x) 58278: 855, // FieldOpts (5x) - 58338: 856, // IndexName (5x) - 58341: 857, // IndexOption (5x) - 58342: 858, // IndexOptionList (5x) - 57438: 859, // infile (5x) - 58368: 860, // LimitOption (5x) - 58380: 861, // LockClause (5x) - 58413: 862, // OptCharsetWithOptBinary (5x) - 58424: 863, // OptNullTreatment (5x) - 58464: 864, // PolicyName (5x) - 58471: 865, // PriorityOpt (5x) - 58511: 866, // SelectLockOpt (5x) - 58518: 867, // SelectStmtIntoOption (5x) - 58605: 868, // TableRefs (5x) - 58631: 869, // UserSpec (5x) - 58130: 870, // Assignment (4x) - 58136: 871, // AuthString (4x) - 58145: 872, // BeginTransactionStmt (4x) - 58147: 873, // BindableStmt (4x) - 58137: 874, // BRIEBooleanOptionName (4x) - 58138: 875, // BRIEIntegerOptionName (4x) - 58139: 876, // BRIEKeywordOptionName (4x) - 58140: 877, // BRIEOption (4x) - 58141: 878, // BRIEOptions (4x) - 58143: 879, // BRIEStringOptionName (4x) - 58157: 880, // ByList (4x) - 58161: 881, // Char (4x) - 58188: 882, // CommitStmt (4x) - 58192: 883, // ConfigItemName (4x) - 58196: 884, // Constraint (4x) - 58286: 885, // FloatOpt (4x) - 58347: 886, // IndexTypeName (4x) - 58376: 887, // LoadDataStmt (4x) - 57490: 888, // option (4x) - 58429: 889, // OptWild (4x) - 57494: 890, // outer (4x) - 58465: 891, // Precision (4x) - 58479: 892, // ReferDef (4x) - 58493: 893, // RestrictOrCascadeOpt (4x) - 58506: 894, // RollbackStmt (4x) - 58509: 895, // RowStmt (4x) - 58526: 896, // SequenceOption (4x) - 58540: 897, // SetStmt (4x) - 57532: 898, // statsExtended (4x) - 58586: 899, // TableAsName (4x) - 58587: 900, // TableAsNameOpt (4x) - 58598: 901, // TableNameOptWild (4x) - 58600: 902, // TableOptimizerHintsOpt (4x) - 58602: 903, // TableOptionList (4x) - 58621: 904, // TransactionChar (4x) - 58632: 905, // UserSpecList (4x) - 58670: 906, // WindowName (4x) - 58127: 907, // AsOfClause (3x) - 58131: 908, // AssignmentList (3x) - 58133: 909, // AttributesOpt (3x) - 58153: 910, // Boolean (3x) - 58181: 911, // ColumnOption (3x) - 58184: 912, // ColumnPosition (3x) - 58189: 913, // CommonTableExpr (3x) - 58210: 914, // CreateTableStmt (3x) - 58218: 915, // DatabaseOptionList (3x) - 58226: 916, // DefaultTrueDistinctOpt (3x) - 58251: 917, // EnforcedOrNot (3x) - 57414: 918, // explain (3x) - 58268: 919, // ExtendedPriv (3x) - 58306: 920, // GeneratedAlways (3x) - 58308: 921, // GlobalScope (3x) - 58312: 922, // GroupByClause (3x) - 58330: 923, // IndexHint (3x) - 58334: 924, // IndexHintType (3x) - 58339: 925, // IndexNameAndTypeOpt (3x) - 57455: 926, // keys (3x) - 58370: 927, // Lines (3x) - 58388: 928, // MaxValueOrExpression (3x) - 58425: 929, // OptOrder (3x) - 58428: 930, // OptTemporary (3x) - 58441: 931, // PartDefOptionList (3x) - 58443: 932, // PartitionDefinition (3x) - 58452: 933, // PasswordExpire (3x) - 58454: 934, // PasswordOrLockOption (3x) - 58463: 935, // PluginNameList (3x) - 58469: 936, // PrimaryOpt (3x) - 58472: 937, // PrivElem (3x) - 58474: 938, // PrivType (3x) - 57500: 939, // procedure (3x) - 58488: 940, // RequireClause (3x) - 58489: 941, // RequireClauseOpt (3x) - 58491: 942, // RequireListElement (3x) - 58505: 943, // RolenameWithoutIdent (3x) - 58498: 944, // RoleOrPrivElem (3x) - 58517: 945, // SelectStmtGroup (3x) - 58534: 946, // SetOprOpt (3x) - 58585: 947, // TableAliasRefList (3x) - 58588: 948, // TableElement (3x) - 58597: 949, // TableNameListOpt2 (3x) - 58613: 950, // TextString (3x) - 58622: 951, // TransactionChars (3x) - 57544: 952, // trigger (3x) - 57548: 953, // unlock (3x) - 57551: 954, // usage (3x) - 58642: 955, // ValuesList (3x) - 58644: 956, // ValuesStmtList (3x) - 58640: 957, // ValueSym (3x) - 58647: 958, // VariableAssignment (3x) - 58667: 959, // WindowFrameStart (3x) - 58103: 960, // AdminStmt (2x) - 58105: 961, // AllColumnsOrPredicateColumnsOpt (2x) - 58107: 962, // AlterDatabaseStmt (2x) - 58108: 963, // AlterImportStmt (2x) - 58109: 964, // AlterInstanceStmt (2x) - 58110: 965, // AlterOrderItem (2x) - 58112: 966, // AlterPolicyStmt (2x) - 58113: 967, // AlterSequenceOption (2x) - 58115: 968, // AlterSequenceStmt (2x) - 58117: 969, // AlterTableSpec (2x) - 58121: 970, // AlterUserStmt (2x) - 58122: 971, // AnalyzeOption (2x) - 58125: 972, // AnalyzeTableStmt (2x) - 58148: 973, // BinlogStmt (2x) - 58142: 974, // BRIEStmt (2x) - 58144: 975, // BRIETables (2x) - 57372: 976, // call (2x) - 58158: 977, // CallStmt (2x) - 58159: 978, // CastType (2x) - 58160: 979, // ChangeStmt (2x) - 58166: 980, // CheckConstraintKeyword (2x) - 58176: 981, // ColumnNameListOpt (2x) - 58179: 982, // ColumnNameOrUserVariable (2x) - 58182: 983, // ColumnOptionList (2x) - 58183: 984, // ColumnOptionListOpt (2x) - 58185: 985, // ColumnSetValue (2x) - 58191: 986, // CompletionTypeWithinTransaction (2x) - 58193: 987, // ConnectionOption (2x) - 58195: 988, // ConnectionOptions (2x) - 58199: 989, // CreateBindingStmt (2x) - 58200: 990, // CreateDatabaseStmt (2x) - 58201: 991, // CreateImportStmt (2x) - 58202: 992, // CreateIndexStmt (2x) - 58203: 993, // CreatePolicyStmt (2x) - 58204: 994, // CreateRoleStmt (2x) - 58206: 995, // CreateSequenceStmt (2x) - 58207: 996, // CreateStatisticsStmt (2x) - 58208: 997, // CreateTableOptionListOpt (2x) - 58211: 998, // CreateUserStmt (2x) - 58213: 999, // CreateViewStmt (2x) - 57392: 1000, // databases (2x) - 58222: 1001, // DeallocateStmt (2x) - 58223: 1002, // DeallocateSym (2x) - 57403: 1003, // describe (2x) - 58234: 1004, // DoStmt (2x) - 58235: 1005, // DropBindingStmt (2x) - 58236: 1006, // DropDatabaseStmt (2x) - 58237: 1007, // DropImportStmt (2x) - 58238: 1008, // DropIndexStmt (2x) - 58239: 1009, // DropPolicyStmt (2x) - 58240: 1010, // DropRoleStmt (2x) - 58241: 1011, // DropSequenceStmt (2x) - 58242: 1012, // DropStatisticsStmt (2x) - 58243: 1013, // DropStatsStmt (2x) - 58244: 1014, // DropTableStmt (2x) - 58245: 1015, // DropUserStmt (2x) - 58246: 1016, // DropViewStmt (2x) - 58247: 1017, // DuplicateOpt (2x) - 58249: 1018, // EmptyStmt (2x) - 58250: 1019, // EncryptionOpt (2x) - 58252: 1020, // EnforcedOrNotOpt (2x) - 58256: 1021, // ErrorHandling (2x) - 58258: 1022, // ExecuteStmt (2x) - 58260: 1023, // ExplainStmt (2x) - 58261: 1024, // ExplainSym (2x) - 58270: 1025, // Field (2x) - 58273: 1026, // FieldItem (2x) - 58280: 1027, // Fields (2x) - 58284: 1028, // FlashbackTableStmt (2x) - 58289: 1029, // FlushStmt (2x) - 58295: 1030, // FuncDatetimePrecList (2x) - 58296: 1031, // FuncDatetimePrecListOpt (2x) - 58309: 1032, // GrantProxyStmt (2x) - 58310: 1033, // GrantRoleStmt (2x) - 58311: 1034, // GrantStmt (2x) - 58313: 1035, // HandleRange (2x) - 58315: 1036, // HashString (2x) - 58317: 1037, // HelpStmt (2x) - 58329: 1038, // IndexAdviseStmt (2x) - 58331: 1039, // IndexHintList (2x) - 58332: 1040, // IndexHintListOpt (2x) - 58337: 1041, // IndexLockAndAlgorithmOpt (2x) - 58350: 1042, // InsertValues (2x) - 58354: 1043, // IntoOpt (2x) - 58360: 1044, // KeyOrIndexOpt (2x) - 57456: 1045, // kill (2x) - 58361: 1046, // KillOrKillTiDB (2x) - 58362: 1047, // KillStmt (2x) - 58367: 1048, // LimitClause (2x) - 57465: 1049, // linear (2x) - 58369: 1050, // LinearOpt (2x) - 58373: 1051, // LoadDataSetItem (2x) - 58377: 1052, // LoadStatsStmt (2x) - 58378: 1053, // LocalOpt (2x) - 58381: 1054, // LockTablesStmt (2x) - 58389: 1055, // MaxValueOrExpressionList (2x) - 58397: 1056, // NowSym (2x) - 58398: 1057, // NowSymFunc (2x) - 58399: 1058, // NowSymOptionFraction (2x) - 58400: 1059, // NumList (2x) - 58403: 1060, // ObjectType (2x) - 57487: 1061, // of (2x) - 58404: 1062, // OfTablesOpt (2x) - 58405: 1063, // OnCommitOpt (2x) - 58406: 1064, // OnDelete (2x) - 58409: 1065, // OnUpdate (2x) - 58414: 1066, // OptCollate (2x) - 58419: 1067, // OptFull (2x) - 58421: 1068, // OptInteger (2x) - 58434: 1069, // OptionalBraces (2x) - 58433: 1070, // OptionLevel (2x) - 58423: 1071, // OptLeadLagInfo (2x) - 58422: 1072, // OptLLDefault (2x) - 58439: 1073, // OuterOpt (2x) - 58444: 1074, // PartitionDefinitionList (2x) - 58445: 1075, // PartitionDefinitionListOpt (2x) - 58451: 1076, // PartitionOpt (2x) - 58453: 1077, // PasswordOpt (2x) - 58455: 1078, // PasswordOrLockOptionList (2x) - 58456: 1079, // PasswordOrLockOptions (2x) - 58460: 1080, // PlacementOptionList (2x) - 58462: 1081, // PlanReplayerStmt (2x) - 58468: 1082, // PreparedStmt (2x) - 58473: 1083, // PrivLevel (2x) - 58476: 1084, // PurgeImportStmt (2x) - 58477: 1085, // QuickOptional (2x) - 58478: 1086, // RecoverTableStmt (2x) - 58480: 1087, // ReferOpt (2x) - 58482: 1088, // RegexpSym (2x) - 58483: 1089, // RenameTableStmt (2x) - 58484: 1090, // RenameUserStmt (2x) - 58486: 1091, // RepeatableOpt (2x) - 58492: 1092, // RestartStmt (2x) - 58494: 1093, // ResumeImportStmt (2x) - 57514: 1094, // revoke (2x) - 58495: 1095, // RevokeRoleStmt (2x) - 58496: 1096, // RevokeStmt (2x) - 58499: 1097, // RoleOrPrivElemList (2x) - 58500: 1098, // RoleSpec (2x) - 58521: 1099, // SelectStmtOpt (2x) - 58524: 1100, // SelectStmtSQLCache (2x) - 58528: 1101, // SetDefaultRoleOpt (2x) - 58529: 1102, // SetDefaultRoleStmt (2x) - 58539: 1103, // SetRoleStmt (2x) - 58542: 1104, // ShowImportStmt (2x) - 58547: 1105, // ShowProfileType (2x) - 58550: 1106, // ShowStmt (2x) - 58551: 1107, // ShowTableAliasOpt (2x) - 58553: 1108, // ShutdownStmt (2x) - 58554: 1109, // SignedLiteral (2x) - 58558: 1110, // SplitOption (2x) - 58559: 1111, // SplitRegionStmt (2x) - 58563: 1112, // Statement (2x) - 58565: 1113, // StatsOptionsOpt (2x) - 58566: 1114, // StatsPersistentVal (2x) - 58567: 1115, // StatsType (2x) - 58568: 1116, // StopImportStmt (2x) - 58575: 1117, // SubPartDefinition (2x) - 58578: 1118, // SubPartitionMethod (2x) - 58583: 1119, // Symbol (2x) - 58589: 1120, // TableElementList (2x) - 58592: 1121, // TableLock (2x) - 58596: 1122, // TableNameListOpt (2x) - 58603: 1123, // TableOrTables (2x) - 58612: 1124, // TablesTerminalSym (2x) - 58610: 1125, // TableToTable (2x) - 58614: 1126, // TextStringList (2x) - 58620: 1127, // TraceableStmt (2x) - 58619: 1128, // TraceStmt (2x) - 58624: 1129, // TruncateTableStmt (2x) - 58627: 1130, // UnlockTablesStmt (2x) - 58633: 1131, // UserToUser (2x) - 58630: 1132, // UseStmt (2x) - 58645: 1133, // Varchar (2x) - 58648: 1134, // VariableAssignmentList (2x) - 58657: 1135, // WhenClause (2x) - 58662: 1136, // WindowDefinition (2x) - 58665: 1137, // WindowFrameBound (2x) - 58672: 1138, // WindowSpec (2x) - 58677: 1139, // WithGrantOptionOpt (2x) - 58678: 1140, // WithList (2x) - 58682: 1141, // Writeable (2x) - 58102: 1142, // AdminShowSlow (1x) - 58111: 1143, // AlterOrderList (1x) - 58114: 1144, // AlterSequenceOptionList (1x) - 58116: 1145, // AlterTablePartitionOpt (1x) - 58118: 1146, // AlterTableSpecList (1x) - 58119: 1147, // AlterTableSpecListOpt (1x) - 58123: 1148, // AnalyzeOptionList (1x) - 58126: 1149, // AnyOrAll (1x) - 58128: 1150, // AsOfClauseOpt (1x) - 58129: 1151, // AsOpt (1x) - 58134: 1152, // AuthOption (1x) - 58135: 1153, // AuthPlugin (1x) - 58146: 1154, // BetweenOrNotOp (1x) - 58150: 1155, // BitValueType (1x) - 58151: 1156, // BlobType (1x) - 58154: 1157, // BooleanType (1x) - 57370: 1158, // both (1x) - 58164: 1159, // CharsetNameOrDefault (1x) - 58165: 1160, // CharsetOpt (1x) - 58167: 1161, // ClearPasswordExpireOptions (1x) - 58171: 1162, // ColumnFormat (1x) - 58173: 1163, // ColumnList (1x) - 58180: 1164, // ColumnNameOrUserVariableList (1x) - 58177: 1165, // ColumnNameOrUserVarListOpt (1x) - 58178: 1166, // ColumnNameOrUserVarListOptWithBrackets (1x) - 58186: 1167, // ColumnSetValueList (1x) - 58190: 1168, // CompareOp (1x) - 58194: 1169, // ConnectionOptionList (1x) - 58197: 1170, // ConstraintElem (1x) - 58205: 1171, // CreateSequenceOptionListOpt (1x) - 58209: 1172, // CreateTableSelectOpt (1x) - 58212: 1173, // CreateViewSelectOpt (1x) - 58219: 1174, // DatabaseOptionListOpt (1x) - 58221: 1175, // DateAndTimeType (1x) - 58216: 1176, // DBNameList (1x) - 58227: 1177, // DefaultValueExpr (1x) - 57409: 1178, // dual (1x) - 58248: 1179, // ElseOpt (1x) - 58253: 1180, // EnforcedOrNotOrNotNullOpt (1x) - 58259: 1181, // ExplainFormatType (1x) - 58267: 1182, // ExpressionOpt (1x) - 58269: 1183, // FetchFirstOpt (1x) - 58271: 1184, // FieldAsName (1x) - 58272: 1185, // FieldAsNameOpt (1x) - 58274: 1186, // FieldItemList (1x) - 58276: 1187, // FieldList (1x) - 58282: 1188, // FirstOrNext (1x) - 58283: 1189, // FixedPointType (1x) - 58285: 1190, // FlashbackToNewName (1x) - 58287: 1191, // FloatingPointType (1x) - 58288: 1192, // FlushOption (1x) - 58291: 1193, // FromDual (1x) - 58293: 1194, // FulltextSearchModifierOpt (1x) - 58294: 1195, // FuncDatetimePrec (1x) - 58307: 1196, // GetFormatSelector (1x) - 58314: 1197, // HandleRangeList (1x) - 58316: 1198, // HavingClause (1x) - 58318: 1199, // IdentList (1x) + 58318: 856, // IdentList (5x) + 58338: 857, // IndexName (5x) + 58341: 858, // IndexOption (5x) + 58342: 859, // IndexOptionList (5x) + 57438: 860, // infile (5x) + 58368: 861, // LimitOption (5x) + 58380: 862, // LockClause (5x) + 58413: 863, // OptCharsetWithOptBinary (5x) + 58424: 864, // OptNullTreatment (5x) + 58464: 865, // PolicyName (5x) + 58471: 866, // PriorityOpt (5x) + 58511: 867, // SelectLockOpt (5x) + 58518: 868, // SelectStmtIntoOption (5x) + 58605: 869, // TableRefs (5x) + 58631: 870, // UserSpec (5x) + 58130: 871, // Assignment (4x) + 58136: 872, // AuthString (4x) + 58145: 873, // BeginTransactionStmt (4x) + 58147: 874, // BindableStmt (4x) + 58137: 875, // BRIEBooleanOptionName (4x) + 58138: 876, // BRIEIntegerOptionName (4x) + 58139: 877, // BRIEKeywordOptionName (4x) + 58140: 878, // BRIEOption (4x) + 58141: 879, // BRIEOptions (4x) + 58143: 880, // BRIEStringOptionName (4x) + 58157: 881, // ByList (4x) + 58161: 882, // Char (4x) + 58188: 883, // CommitStmt (4x) + 58192: 884, // ConfigItemName (4x) + 58196: 885, // Constraint (4x) + 58286: 886, // FloatOpt (4x) + 58347: 887, // IndexTypeName (4x) + 58376: 888, // LoadDataStmt (4x) + 57490: 889, // option (4x) + 58429: 890, // OptWild (4x) + 57494: 891, // outer (4x) + 58465: 892, // Precision (4x) + 58479: 893, // ReferDef (4x) + 58493: 894, // RestrictOrCascadeOpt (4x) + 58506: 895, // RollbackStmt (4x) + 58509: 896, // RowStmt (4x) + 58526: 897, // SequenceOption (4x) + 58540: 898, // SetStmt (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) + 58621: 905, // TransactionChar (4x) + 58632: 906, // UserSpecList (4x) + 58670: 907, // WindowName (4x) + 58127: 908, // AsOfClause (3x) + 58131: 909, // AssignmentList (3x) + 58133: 910, // AttributesOpt (3x) + 58153: 911, // Boolean (3x) + 58181: 912, // ColumnOption (3x) + 58184: 913, // ColumnPosition (3x) + 58189: 914, // CommonTableExpr (3x) + 58210: 915, // CreateTableStmt (3x) + 58218: 916, // DatabaseOptionList (3x) + 58226: 917, // DefaultTrueDistinctOpt (3x) + 58251: 918, // EnforcedOrNot (3x) + 57414: 919, // explain (3x) + 58268: 920, // ExtendedPriv (3x) + 58306: 921, // GeneratedAlways (3x) + 58308: 922, // GlobalScope (3x) + 58312: 923, // GroupByClause (3x) + 58330: 924, // IndexHint (3x) + 58334: 925, // IndexHintType (3x) + 58339: 926, // IndexNameAndTypeOpt (3x) + 57455: 927, // keys (3x) + 58370: 928, // Lines (3x) + 58388: 929, // MaxValueOrExpression (3x) + 58425: 930, // OptOrder (3x) + 58428: 931, // OptTemporary (3x) + 58441: 932, // PartDefOptionList (3x) + 58443: 933, // PartitionDefinition (3x) + 58452: 934, // PasswordExpire (3x) + 58454: 935, // PasswordOrLockOption (3x) + 58463: 936, // PluginNameList (3x) + 58469: 937, // PrimaryOpt (3x) + 58472: 938, // PrivElem (3x) + 58474: 939, // PrivType (3x) + 57500: 940, // procedure (3x) + 58488: 941, // RequireClause (3x) + 58489: 942, // RequireClauseOpt (3x) + 58491: 943, // RequireListElement (3x) + 58505: 944, // RolenameWithoutIdent (3x) + 58498: 945, // RoleOrPrivElem (3x) + 58517: 946, // SelectStmtGroup (3x) + 58534: 947, // SetOprOpt (3x) + 58585: 948, // TableAliasRefList (3x) + 58588: 949, // TableElement (3x) + 58597: 950, // TableNameListOpt2 (3x) + 58613: 951, // TextString (3x) + 58622: 952, // TransactionChars (3x) + 57544: 953, // trigger (3x) + 57548: 954, // unlock (3x) + 57551: 955, // usage (3x) + 58642: 956, // ValuesList (3x) + 58644: 957, // ValuesStmtList (3x) + 58640: 958, // ValueSym (3x) + 58647: 959, // VariableAssignment (3x) + 58667: 960, // WindowFrameStart (3x) + 58103: 961, // AdminStmt (2x) + 58105: 962, // AllColumnsOrPredicateColumnsOpt (2x) + 58107: 963, // AlterDatabaseStmt (2x) + 58108: 964, // AlterImportStmt (2x) + 58109: 965, // AlterInstanceStmt (2x) + 58110: 966, // AlterOrderItem (2x) + 58112: 967, // AlterPolicyStmt (2x) + 58113: 968, // AlterSequenceOption (2x) + 58115: 969, // AlterSequenceStmt (2x) + 58117: 970, // AlterTableSpec (2x) + 58121: 971, // AlterUserStmt (2x) + 58122: 972, // AnalyzeOption (2x) + 58125: 973, // AnalyzeTableStmt (2x) + 58148: 974, // BinlogStmt (2x) + 58142: 975, // BRIEStmt (2x) + 58144: 976, // BRIETables (2x) + 57372: 977, // call (2x) + 58158: 978, // CallStmt (2x) + 58159: 979, // CastType (2x) + 58160: 980, // ChangeStmt (2x) + 58166: 981, // CheckConstraintKeyword (2x) + 58176: 982, // ColumnNameListOpt (2x) + 58179: 983, // ColumnNameOrUserVariable (2x) + 58182: 984, // ColumnOptionList (2x) + 58183: 985, // ColumnOptionListOpt (2x) + 58185: 986, // ColumnSetValue (2x) + 58191: 987, // CompletionTypeWithinTransaction (2x) + 58193: 988, // ConnectionOption (2x) + 58195: 989, // ConnectionOptions (2x) + 58199: 990, // CreateBindingStmt (2x) + 58200: 991, // CreateDatabaseStmt (2x) + 58201: 992, // CreateImportStmt (2x) + 58202: 993, // CreateIndexStmt (2x) + 58203: 994, // CreatePolicyStmt (2x) + 58204: 995, // CreateRoleStmt (2x) + 58206: 996, // CreateSequenceStmt (2x) + 58207: 997, // CreateStatisticsStmt (2x) + 58208: 998, // CreateTableOptionListOpt (2x) + 58211: 999, // CreateUserStmt (2x) + 58213: 1000, // CreateViewStmt (2x) + 57392: 1001, // databases (2x) + 58222: 1002, // DeallocateStmt (2x) + 58223: 1003, // DeallocateSym (2x) + 57403: 1004, // describe (2x) + 58234: 1005, // DoStmt (2x) + 58235: 1006, // DropBindingStmt (2x) + 58236: 1007, // DropDatabaseStmt (2x) + 58237: 1008, // DropImportStmt (2x) + 58238: 1009, // DropIndexStmt (2x) + 58239: 1010, // DropPolicyStmt (2x) + 58240: 1011, // DropRoleStmt (2x) + 58241: 1012, // DropSequenceStmt (2x) + 58242: 1013, // DropStatisticsStmt (2x) + 58243: 1014, // DropStatsStmt (2x) + 58244: 1015, // DropTableStmt (2x) + 58245: 1016, // DropUserStmt (2x) + 58246: 1017, // DropViewStmt (2x) + 58247: 1018, // DuplicateOpt (2x) + 58249: 1019, // EmptyStmt (2x) + 58250: 1020, // EncryptionOpt (2x) + 58252: 1021, // EnforcedOrNotOpt (2x) + 58256: 1022, // ErrorHandling (2x) + 58258: 1023, // ExecuteStmt (2x) + 58260: 1024, // ExplainStmt (2x) + 58261: 1025, // ExplainSym (2x) + 58270: 1026, // Field (2x) + 58273: 1027, // FieldItem (2x) + 58280: 1028, // Fields (2x) + 58284: 1029, // FlashbackTableStmt (2x) + 58289: 1030, // FlushStmt (2x) + 58295: 1031, // FuncDatetimePrecList (2x) + 58296: 1032, // FuncDatetimePrecListOpt (2x) + 58309: 1033, // GrantProxyStmt (2x) + 58310: 1034, // GrantRoleStmt (2x) + 58311: 1035, // GrantStmt (2x) + 58313: 1036, // HandleRange (2x) + 58315: 1037, // HashString (2x) + 58317: 1038, // HelpStmt (2x) + 58329: 1039, // IndexAdviseStmt (2x) + 58331: 1040, // IndexHintList (2x) + 58332: 1041, // IndexHintListOpt (2x) + 58337: 1042, // IndexLockAndAlgorithmOpt (2x) + 58350: 1043, // InsertValues (2x) + 58354: 1044, // IntoOpt (2x) + 58360: 1045, // KeyOrIndexOpt (2x) + 57456: 1046, // kill (2x) + 58361: 1047, // KillOrKillTiDB (2x) + 58362: 1048, // KillStmt (2x) + 58367: 1049, // LimitClause (2x) + 57465: 1050, // linear (2x) + 58369: 1051, // LinearOpt (2x) + 58373: 1052, // LoadDataSetItem (2x) + 58377: 1053, // LoadStatsStmt (2x) + 58378: 1054, // LocalOpt (2x) + 58381: 1055, // LockTablesStmt (2x) + 58389: 1056, // MaxValueOrExpressionList (2x) + 58397: 1057, // NowSym (2x) + 58398: 1058, // NowSymFunc (2x) + 58399: 1059, // NowSymOptionFraction (2x) + 58400: 1060, // NumList (2x) + 58403: 1061, // ObjectType (2x) + 57487: 1062, // of (2x) + 58404: 1063, // OfTablesOpt (2x) + 58405: 1064, // OnCommitOpt (2x) + 58406: 1065, // OnDelete (2x) + 58409: 1066, // OnUpdate (2x) + 58414: 1067, // OptCollate (2x) + 58419: 1068, // OptFull (2x) + 58421: 1069, // OptInteger (2x) + 58434: 1070, // OptionalBraces (2x) + 58433: 1071, // OptionLevel (2x) + 58423: 1072, // OptLeadLagInfo (2x) + 58422: 1073, // OptLLDefault (2x) + 58439: 1074, // OuterOpt (2x) + 58444: 1075, // PartitionDefinitionList (2x) + 58445: 1076, // PartitionDefinitionListOpt (2x) + 58451: 1077, // PartitionOpt (2x) + 58453: 1078, // PasswordOpt (2x) + 58455: 1079, // PasswordOrLockOptionList (2x) + 58456: 1080, // PasswordOrLockOptions (2x) + 58460: 1081, // PlacementOptionList (2x) + 58462: 1082, // PlanReplayerStmt (2x) + 58468: 1083, // PreparedStmt (2x) + 58473: 1084, // PrivLevel (2x) + 58476: 1085, // PurgeImportStmt (2x) + 58477: 1086, // QuickOptional (2x) + 58478: 1087, // RecoverTableStmt (2x) + 58480: 1088, // ReferOpt (2x) + 58482: 1089, // RegexpSym (2x) + 58483: 1090, // RenameTableStmt (2x) + 58484: 1091, // RenameUserStmt (2x) + 58486: 1092, // RepeatableOpt (2x) + 58492: 1093, // RestartStmt (2x) + 58494: 1094, // ResumeImportStmt (2x) + 57514: 1095, // revoke (2x) + 58495: 1096, // RevokeRoleStmt (2x) + 58496: 1097, // RevokeStmt (2x) + 58499: 1098, // RoleOrPrivElemList (2x) + 58500: 1099, // RoleSpec (2x) + 58521: 1100, // SelectStmtOpt (2x) + 58524: 1101, // SelectStmtSQLCache (2x) + 58528: 1102, // SetDefaultRoleOpt (2x) + 58529: 1103, // SetDefaultRoleStmt (2x) + 58539: 1104, // SetRoleStmt (2x) + 58542: 1105, // ShowImportStmt (2x) + 58547: 1106, // ShowProfileType (2x) + 58550: 1107, // ShowStmt (2x) + 58551: 1108, // ShowTableAliasOpt (2x) + 58553: 1109, // ShutdownStmt (2x) + 58554: 1110, // SignedLiteral (2x) + 58558: 1111, // SplitOption (2x) + 58559: 1112, // SplitRegionStmt (2x) + 58563: 1113, // Statement (2x) + 58565: 1114, // StatsOptionsOpt (2x) + 58566: 1115, // StatsPersistentVal (2x) + 58567: 1116, // StatsType (2x) + 58568: 1117, // StopImportStmt (2x) + 58575: 1118, // SubPartDefinition (2x) + 58578: 1119, // SubPartitionMethod (2x) + 58583: 1120, // Symbol (2x) + 58589: 1121, // TableElementList (2x) + 58592: 1122, // TableLock (2x) + 58596: 1123, // TableNameListOpt (2x) + 58603: 1124, // TableOrTables (2x) + 58612: 1125, // TablesTerminalSym (2x) + 58610: 1126, // TableToTable (2x) + 58614: 1127, // TextStringList (2x) + 58620: 1128, // TraceableStmt (2x) + 58619: 1129, // TraceStmt (2x) + 58624: 1130, // TruncateTableStmt (2x) + 58627: 1131, // UnlockTablesStmt (2x) + 58633: 1132, // UserToUser (2x) + 58630: 1133, // UseStmt (2x) + 58645: 1134, // Varchar (2x) + 58648: 1135, // VariableAssignmentList (2x) + 58657: 1136, // WhenClause (2x) + 58662: 1137, // WindowDefinition (2x) + 58665: 1138, // WindowFrameBound (2x) + 58672: 1139, // WindowSpec (2x) + 58677: 1140, // WithGrantOptionOpt (2x) + 58678: 1141, // WithList (2x) + 58682: 1142, // Writeable (2x) + 58102: 1143, // AdminShowSlow (1x) + 58111: 1144, // AlterOrderList (1x) + 58114: 1145, // AlterSequenceOptionList (1x) + 58116: 1146, // AlterTablePartitionOpt (1x) + 58118: 1147, // AlterTableSpecList (1x) + 58119: 1148, // AlterTableSpecListOpt (1x) + 58123: 1149, // AnalyzeOptionList (1x) + 58126: 1150, // AnyOrAll (1x) + 58128: 1151, // AsOfClauseOpt (1x) + 58129: 1152, // AsOpt (1x) + 58134: 1153, // AuthOption (1x) + 58135: 1154, // AuthPlugin (1x) + 58146: 1155, // BetweenOrNotOp (1x) + 58150: 1156, // BitValueType (1x) + 58151: 1157, // BlobType (1x) + 58154: 1158, // BooleanType (1x) + 57370: 1159, // both (1x) + 58164: 1160, // CharsetNameOrDefault (1x) + 58165: 1161, // CharsetOpt (1x) + 58167: 1162, // ClearPasswordExpireOptions (1x) + 58171: 1163, // ColumnFormat (1x) + 58173: 1164, // ColumnList (1x) + 58180: 1165, // ColumnNameOrUserVariableList (1x) + 58177: 1166, // ColumnNameOrUserVarListOpt (1x) + 58178: 1167, // ColumnNameOrUserVarListOptWithBrackets (1x) + 58186: 1168, // ColumnSetValueList (1x) + 58190: 1169, // CompareOp (1x) + 58194: 1170, // ConnectionOptionList (1x) + 58197: 1171, // ConstraintElem (1x) + 58205: 1172, // CreateSequenceOptionListOpt (1x) + 58209: 1173, // CreateTableSelectOpt (1x) + 58212: 1174, // CreateViewSelectOpt (1x) + 58219: 1175, // DatabaseOptionListOpt (1x) + 58221: 1176, // DateAndTimeType (1x) + 58216: 1177, // DBNameList (1x) + 58227: 1178, // DefaultValueExpr (1x) + 57409: 1179, // dual (1x) + 58248: 1180, // ElseOpt (1x) + 58253: 1181, // EnforcedOrNotOrNotNullOpt (1x) + 58259: 1182, // ExplainFormatType (1x) + 58267: 1183, // ExpressionOpt (1x) + 58269: 1184, // FetchFirstOpt (1x) + 58271: 1185, // FieldAsName (1x) + 58272: 1186, // FieldAsNameOpt (1x) + 58274: 1187, // FieldItemList (1x) + 58276: 1188, // FieldList (1x) + 58282: 1189, // FirstOrNext (1x) + 58283: 1190, // FixedPointType (1x) + 58285: 1191, // FlashbackToNewName (1x) + 58287: 1192, // FloatingPointType (1x) + 58288: 1193, // FlushOption (1x) + 58291: 1194, // FromDual (1x) + 58293: 1195, // FulltextSearchModifierOpt (1x) + 58294: 1196, // FuncDatetimePrec (1x) + 58307: 1197, // GetFormatSelector (1x) + 58314: 1198, // HandleRangeList (1x) + 58316: 1199, // HavingClause (1x) 58319: 1200, // IdentListWithParenOpt (1x) 58323: 1201, // IfNotRunning (1x) 58324: 1202, // IfRunning (1x) @@ -2955,7 +2955,6 @@ var ( "DistinctKwd", "IfNotExists", "OptFieldLen", - "ColumnNameList", "DistinctOpt", "enclosed", "InsertIntoStmt", @@ -2979,6 +2978,7 @@ var ( "FromOrIn", "TimestampUnit", "CharsetName", + "ColumnNameList", "DeleteFromStmt", "NotSym", "OrderByOptional", @@ -3040,6 +3040,7 @@ var ( "ColumnKeywordOpt", "FieldOpt", "FieldOpts", + "IdentList", "IndexName", "IndexOption", "IndexOptionList", @@ -3383,7 +3384,6 @@ var ( "GetFormatSelector", "HandleRangeList", "HavingClause", - "IdentList", "IdentListWithParenOpt", "IfNotRunning", "IfRunning", @@ -3554,9 +3554,9 @@ var ( {808, 6}, {808, 8}, {808, 10}, - {1080, 1}, - {1080, 2}, - {1080, 3}, + {1081, 1}, + {1081, 2}, + {1081, 3}, {758, 3}, {758, 3}, {758, 3}, @@ -3574,72 +3574,72 @@ var ( {762, 4}, {762, 4}, {762, 4}, - {909, 3}, - {909, 3}, - {1113, 3}, - {1113, 3}, - {1145, 1}, - {1145, 2}, - {1145, 4}, - {1145, 3}, - {1145, 3}, + {910, 3}, + {910, 3}, + {1114, 3}, + {1114, 3}, + {1146, 1}, + {1146, 2}, + {1146, 4}, + {1146, 3}, + {1146, 3}, {1221, 0}, {1221, 3}, - {969, 1}, - {969, 5}, - {969, 5}, - {969, 5}, - {969, 5}, - {969, 6}, - {969, 2}, - {969, 5}, - {969, 6}, - {969, 8}, - {969, 1}, - {969, 1}, - {969, 3}, - {969, 4}, - {969, 5}, - {969, 3}, - {969, 4}, - {969, 4}, - {969, 7}, - {969, 3}, - {969, 4}, - {969, 4}, - {969, 4}, - {969, 4}, - {969, 2}, - {969, 2}, - {969, 4}, - {969, 4}, - {969, 5}, - {969, 3}, - {969, 2}, - {969, 2}, - {969, 5}, - {969, 6}, - {969, 6}, - {969, 8}, - {969, 5}, - {969, 5}, - {969, 3}, - {969, 3}, - {969, 3}, - {969, 5}, - {969, 1}, - {969, 1}, - {969, 1}, - {969, 1}, - {969, 2}, - {969, 2}, - {969, 1}, - {969, 1}, - {969, 4}, - {969, 3}, - {969, 4}, - {969, 1}, - {969, 1}, + {970, 1}, + {970, 5}, + {970, 5}, + {970, 5}, + {970, 5}, + {970, 6}, + {970, 2}, + {970, 5}, + {970, 6}, + {970, 8}, + {970, 1}, + {970, 1}, + {970, 3}, + {970, 4}, + {970, 5}, + {970, 3}, + {970, 4}, + {970, 4}, + {970, 7}, + {970, 3}, + {970, 4}, + {970, 4}, + {970, 4}, + {970, 4}, + {970, 2}, + {970, 2}, + {970, 4}, + {970, 4}, + {970, 5}, + {970, 3}, + {970, 2}, + {970, 2}, + {970, 5}, + {970, 6}, + {970, 6}, + {970, 8}, + {970, 5}, + {970, 5}, + {970, 3}, + {970, 3}, + {970, 3}, + {970, 5}, + {970, 1}, + {970, 1}, + {970, 1}, + {970, 1}, + {970, 2}, + {970, 2}, + {970, 1}, + {970, 1}, + {970, 4}, + {970, 3}, + {970, 4}, + {970, 1}, + {970, 1}, {1257, 0}, {1257, 5}, {818, 1}, @@ -3655,89 +3655,89 @@ var ( {850, 3}, {850, 3}, {850, 3}, - {861, 3}, - {861, 3}, - {1141, 2}, - {1141, 2}, + {862, 3}, + {862, 3}, + {1142, 2}, + {1142, 2}, {813, 1}, {813, 1}, - {1044, 0}, - {1044, 1}, + {1045, 0}, + {1045, 1}, {853, 0}, {853, 1}, - {912, 0}, - {912, 1}, - {912, 2}, - {1147, 0}, + {913, 0}, + {913, 1}, + {913, 2}, + {1148, 0}, + {1148, 1}, {1147, 1}, - {1146, 1}, - {1146, 3}, - {775, 1}, - {775, 3}, + {1147, 3}, + {774, 1}, + {774, 3}, {819, 0}, {819, 1}, {819, 2}, - {1119, 1}, - {1089, 3}, + {1120, 1}, + {1090, 3}, {1296, 1}, {1296, 3}, - {1125, 3}, - {1090, 3}, + {1126, 3}, + {1091, 3}, {1301, 1}, {1301, 3}, - {1131, 3}, - {1086, 5}, - {1086, 3}, - {1086, 4}, - {1028, 4}, - {1190, 0}, - {1190, 2}, + {1132, 3}, + {1087, 5}, + {1087, 3}, + {1087, 4}, + {1029, 4}, + {1191, 0}, + {1191, 2}, + {1112, 6}, + {1112, 8}, {1111, 6}, - {1111, 8}, - {1110, 6}, - {1110, 2}, + {1111, 2}, {1275, 0}, {1275, 2}, {1275, 1}, {1275, 3}, - {972, 5}, - {972, 6}, - {972, 7}, - {972, 7}, - {972, 8}, - {972, 9}, - {972, 8}, - {972, 7}, - {972, 6}, - {972, 8}, - {961, 0}, - {961, 2}, - {961, 2}, - {789, 0}, - {789, 2}, - {1148, 1}, - {1148, 3}, - {971, 2}, - {971, 2}, - {971, 3}, - {971, 3}, - {971, 2}, - {971, 2}, - {870, 3}, - {908, 1}, - {908, 3}, + {973, 5}, + {973, 6}, + {973, 7}, + {973, 7}, + {973, 8}, + {973, 9}, + {973, 8}, + {973, 7}, + {973, 6}, + {973, 8}, + {962, 0}, + {962, 2}, + {962, 2}, + {788, 0}, + {788, 2}, + {1149, 1}, + {1149, 3}, + {972, 2}, + {972, 2}, + {972, 3}, + {972, 3}, + {972, 2}, + {972, 2}, + {871, 3}, + {909, 1}, + {909, 3}, {1328, 0}, {1328, 1}, - {872, 1}, - {872, 2}, - {872, 2}, - {872, 2}, - {872, 4}, - {872, 5}, - {872, 6}, - {872, 4}, - {872, 5}, - {973, 2}, + {873, 1}, + {873, 2}, + {873, 2}, + {873, 2}, + {873, 4}, + {873, 5}, + {873, 6}, + {873, 4}, + {873, 5}, + {974, 2}, {1329, 1}, {1329, 3}, {831, 3}, @@ -3745,158 +3745,158 @@ var ( {729, 1}, {729, 3}, {729, 5}, - {771, 1}, - {771, 3}, - {981, 0}, - {981, 1}, + {794, 1}, + {794, 3}, + {982, 0}, + {982, 1}, {1200, 0}, {1200, 3}, - {1199, 1}, - {1199, 3}, - {1165, 0}, - {1165, 1}, - {1164, 1}, - {1164, 3}, - {982, 1}, - {982, 1}, + {856, 1}, + {856, 3}, {1166, 0}, - {1166, 3}, - {882, 1}, - {882, 2}, - {936, 0}, - {936, 1}, + {1166, 1}, + {1165, 1}, + {1165, 3}, + {983, 1}, + {983, 1}, + {1167, 0}, + {1167, 3}, + {883, 1}, + {883, 2}, + {937, 0}, + {937, 1}, {796, 1}, {796, 1}, - {917, 1}, - {917, 2}, - {1020, 0}, - {1020, 1}, - {1180, 2}, - {1180, 1}, - {911, 2}, - {911, 1}, - {911, 1}, - {911, 2}, - {911, 3}, - {911, 1}, - {911, 2}, - {911, 2}, - {911, 3}, - {911, 3}, - {911, 2}, - {911, 6}, - {911, 6}, - {911, 1}, - {911, 2}, - {911, 2}, - {911, 2}, - {911, 2}, + {918, 1}, + {918, 2}, + {1021, 0}, + {1021, 1}, + {1181, 2}, + {1181, 1}, + {912, 2}, + {912, 1}, + {912, 1}, + {912, 2}, + {912, 3}, + {912, 1}, + {912, 2}, + {912, 2}, + {912, 3}, + {912, 3}, + {912, 2}, + {912, 6}, + {912, 6}, + {912, 1}, + {912, 2}, + {912, 2}, + {912, 2}, + {912, 2}, {1281, 1}, {1281, 1}, {1281, 1}, - {1162, 1}, - {1162, 1}, - {1162, 1}, - {920, 0}, - {920, 2}, + {1163, 1}, + {1163, 1}, + {1163, 1}, + {921, 0}, + {921, 2}, {1313, 0}, {1313, 1}, {1313, 1}, - {983, 1}, - {983, 2}, - {984, 0}, {984, 1}, - {1170, 7}, - {1170, 7}, - {1170, 7}, - {1170, 7}, - {1170, 8}, - {1170, 5}, + {984, 2}, + {985, 0}, + {985, 1}, + {1171, 7}, + {1171, 7}, + {1171, 7}, + {1171, 7}, + {1171, 8}, + {1171, 5}, {1224, 2}, {1224, 2}, {1224, 2}, {1225, 0}, {1225, 1}, - {892, 5}, - {1064, 3}, + {893, 5}, {1065, 3}, + {1066, 3}, {1231, 0}, {1231, 1}, {1231, 1}, {1231, 2}, {1231, 2}, - {1087, 1}, - {1087, 1}, - {1087, 2}, - {1087, 2}, - {1087, 2}, - {1177, 1}, - {1177, 1}, - {1177, 1}, - {1058, 1}, - {1058, 3}, - {1058, 4}, + {1088, 1}, + {1088, 1}, + {1088, 2}, + {1088, 2}, + {1088, 2}, + {1178, 1}, + {1178, 1}, + {1178, 1}, + {1059, 1}, + {1059, 3}, + {1059, 4}, {701, 4}, {701, 4}, + {1058, 1}, + {1058, 1}, + {1058, 1}, + {1058, 1}, {1057, 1}, {1057, 1}, {1057, 1}, - {1057, 1}, - {1056, 1}, - {1056, 1}, - {1056, 1}, - {1109, 1}, - {1109, 2}, - {1109, 2}, + {1110, 1}, + {1110, 2}, + {1110, 2}, {805, 1}, {805, 1}, {805, 1}, - {1115, 1}, - {1115, 1}, - {1115, 1}, - {996, 12}, - {1012, 3}, - {992, 13}, + {1116, 1}, + {1116, 1}, + {1116, 1}, + {997, 12}, + {1013, 3}, + {993, 13}, {1207, 0}, {1207, 3}, {822, 1}, {822, 3}, {812, 3}, {812, 4}, - {1041, 0}, - {1041, 1}, - {1041, 1}, - {1041, 2}, - {1041, 2}, + {1042, 0}, + {1042, 1}, + {1042, 1}, + {1042, 2}, + {1042, 2}, {1206, 0}, {1206, 1}, {1206, 1}, {1206, 1}, - {962, 4}, - {962, 3}, - {990, 5}, + {963, 4}, + {963, 3}, + {991, 5}, {801, 1}, - {864, 1}, + {865, 1}, {832, 4}, {832, 4}, {832, 4}, {832, 2}, {832, 1}, - {1174, 0}, - {1174, 1}, - {915, 1}, - {915, 2}, - {914, 12}, - {914, 7}, - {1063, 0}, - {1063, 4}, - {1063, 4}, - {780, 0}, - {780, 1}, - {1076, 0}, - {1076, 6}, - {1118, 6}, - {1118, 5}, + {1175, 0}, + {1175, 1}, + {916, 1}, + {916, 2}, + {915, 12}, + {915, 7}, + {1064, 0}, + {1064, 4}, + {1064, 4}, + {779, 0}, + {779, 1}, + {1077, 0}, + {1077, 6}, + {1119, 6}, + {1119, 5}, {1247, 0}, {1247, 3}, {1248, 1}, @@ -3907,26 +3907,26 @@ var ( {1248, 4}, {1248, 3}, {1248, 1}, - {1050, 0}, - {1050, 1}, + {1051, 0}, + {1051, 1}, {1289, 0}, {1289, 4}, {1288, 0}, {1288, 2}, {1249, 0}, {1249, 2}, - {1075, 0}, + {1076, 0}, + {1076, 3}, + {1075, 1}, {1075, 3}, - {1074, 1}, - {1074, 3}, - {932, 5}, + {933, 5}, {1287, 0}, {1287, 3}, {1286, 1}, {1286, 3}, - {1117, 3}, - {931, 0}, - {931, 2}, + {1118, 3}, + {932, 0}, + {932, 2}, {798, 3}, {798, 3}, {798, 4}, @@ -3945,23 +3945,23 @@ var ( {1246, 5}, {1246, 1}, {1246, 1}, - {1017, 0}, - {1017, 1}, - {1017, 1}, - {1151, 0}, - {1151, 1}, - {1172, 0}, - {1172, 1}, - {1172, 1}, - {1172, 1}, - {1172, 1}, + {1018, 0}, + {1018, 1}, + {1018, 1}, + {1152, 0}, + {1152, 1}, + {1173, 0}, {1173, 1}, {1173, 1}, {1173, 1}, {1173, 1}, + {1174, 1}, + {1174, 1}, + {1174, 1}, + {1174, 1}, {1217, 2}, {1217, 4}, - {999, 11}, + {1000, 11}, {1244, 0}, {1244, 2}, {1306, 0}, @@ -3976,133 +3976,133 @@ var ( {1310, 1}, {1309, 0}, {1309, 3}, - {1163, 1}, - {1163, 3}, + {1164, 1}, + {1164, 3}, {1307, 0}, {1307, 4}, {1307, 4}, - {1004, 2}, + {1005, 2}, {767, 13}, {767, 9}, - {790, 10}, + {789, 10}, {795, 1}, {795, 1}, {795, 2}, {795, 2}, {833, 1}, - {1006, 4}, - {1008, 7}, - {1014, 6}, - {930, 0}, - {930, 1}, - {930, 2}, - {1016, 4}, - {1016, 6}, - {1015, 3}, - {1015, 5}, - {1010, 3}, - {1010, 5}, - {1013, 3}, - {1013, 5}, - {1013, 4}, - {893, 0}, - {893, 1}, - {893, 1}, - {1123, 1}, - {1123, 1}, + {1007, 4}, + {1009, 7}, + {1015, 6}, + {931, 0}, + {931, 1}, + {931, 2}, + {1017, 4}, + {1017, 6}, + {1016, 3}, + {1016, 5}, + {1011, 3}, + {1011, 5}, + {1014, 3}, + {1014, 5}, + {1014, 4}, + {894, 0}, + {894, 1}, + {894, 1}, + {1124, 1}, + {1124, 1}, {723, 0}, {723, 1}, - {1018, 0}, - {1128, 2}, - {1128, 5}, - {1024, 1}, - {1024, 1}, - {1024, 1}, - {1023, 2}, - {1023, 3}, - {1023, 2}, - {1023, 4}, - {1023, 7}, - {1023, 5}, - {1023, 7}, - {1023, 5}, - {1023, 3}, - {1181, 1}, - {1181, 1}, - {1181, 1}, - {1181, 1}, - {1181, 1}, - {1181, 1}, - {974, 5}, - {974, 5}, - {975, 2}, - {975, 2}, - {975, 2}, - {1176, 1}, - {1176, 3}, - {878, 0}, - {878, 2}, - {875, 1}, - {875, 1}, - {874, 1}, - {874, 1}, - {874, 1}, - {874, 1}, - {874, 1}, - {874, 1}, - {874, 1}, - {874, 1}, - {879, 1}, - {879, 1}, - {879, 1}, - {879, 1}, + {1019, 0}, + {1129, 2}, + {1129, 5}, + {1025, 1}, + {1025, 1}, + {1025, 1}, + {1024, 2}, + {1024, 3}, + {1024, 2}, + {1024, 4}, + {1024, 7}, + {1024, 5}, + {1024, 7}, + {1024, 5}, + {1024, 3}, + {1182, 1}, + {1182, 1}, + {1182, 1}, + {1182, 1}, + {1182, 1}, + {1182, 1}, + {975, 5}, + {975, 5}, + {976, 2}, + {976, 2}, + {976, 2}, + {1177, 1}, + {1177, 3}, + {879, 0}, + {879, 2}, {876, 1}, {876, 1}, - {876, 2}, - {877, 3}, - {877, 3}, - {877, 3}, - {877, 3}, - {877, 5}, - {877, 3}, - {877, 3}, - {877, 3}, - {877, 3}, - {877, 6}, - {877, 3}, - {877, 3}, - {877, 3}, - {877, 3}, - {877, 3}, - {877, 3}, + {875, 1}, + {875, 1}, + {875, 1}, + {875, 1}, + {875, 1}, + {875, 1}, + {875, 1}, + {875, 1}, + {880, 1}, + {880, 1}, + {880, 1}, + {880, 1}, + {877, 1}, + {877, 1}, + {877, 2}, + {878, 3}, + {878, 3}, + {878, 3}, + {878, 3}, + {878, 5}, + {878, 3}, + {878, 3}, + {878, 3}, + {878, 3}, + {878, 6}, + {878, 3}, + {878, 3}, + {878, 3}, + {878, 3}, + {878, 3}, + {878, 3}, {730, 1}, {748, 1}, {722, 1}, - {910, 1}, - {910, 1}, - {910, 1}, - {1070, 1}, - {1070, 1}, - {1070, 1}, - {1084, 3}, - {991, 8}, - {1116, 4}, - {1093, 4}, - {963, 6}, - {1007, 4}, - {1104, 5}, + {911, 1}, + {911, 1}, + {911, 1}, + {1071, 1}, + {1071, 1}, + {1071, 1}, + {1085, 3}, + {992, 8}, + {1117, 4}, + {1094, 4}, + {964, 6}, + {1008, 4}, + {1105, 5}, {1202, 0}, {1202, 2}, {1201, 0}, {1201, 3}, {1235, 0}, {1235, 1}, - {1021, 0}, - {1021, 1}, - {1021, 2}, - {1021, 2}, - {1021, 2}, - {1021, 2}, + {1022, 0}, + {1022, 1}, + {1022, 2}, + {1022, 2}, + {1022, 2}, + {1022, 2}, {1204, 0}, {1204, 3}, {1204, 3}, @@ -4116,41 +4116,41 @@ var ( {719, 3}, {719, 3}, {719, 1}, - {928, 1}, - {928, 1}, - {1194, 0}, - {1194, 4}, - {1194, 7}, - {1194, 3}, - {1194, 3}, + {929, 1}, + {929, 1}, + {1195, 0}, + {1195, 4}, + {1195, 7}, + {1195, 3}, + {1195, 3}, {721, 1}, {721, 1}, {720, 1}, {720, 1}, {761, 1}, {761, 3}, - {1055, 1}, - {1055, 3}, + {1056, 1}, + {1056, 3}, {811, 0}, {811, 1}, - {1031, 0}, + {1032, 0}, + {1032, 1}, {1031, 1}, - {1030, 1}, {718, 3}, {718, 3}, {718, 4}, {718, 5}, {718, 1}, - {1168, 1}, - {1168, 1}, - {1168, 1}, - {1168, 1}, - {1168, 1}, - {1168, 1}, - {1168, 1}, - {1168, 1}, - {1154, 1}, - {1154, 2}, + {1169, 1}, + {1169, 1}, + {1169, 1}, + {1169, 1}, + {1169, 1}, + {1169, 1}, + {1169, 1}, + {1169, 1}, + {1155, 1}, + {1155, 2}, {1213, 1}, {1213, 2}, {1209, 1}, @@ -4159,63 +4159,63 @@ var ( {1216, 2}, {1256, 1}, {1256, 2}, - {1149, 1}, - {1149, 1}, - {1149, 1}, + {1150, 1}, + {1150, 1}, + {1150, 1}, {717, 5}, {717, 3}, {717, 5}, {717, 4}, {717, 3}, {717, 1}, - {1088, 1}, - {1088, 1}, + {1089, 1}, + {1089, 1}, {1215, 0}, {1215, 2}, - {1025, 1}, - {1025, 3}, - {1025, 5}, - {1025, 2}, - {1185, 0}, + {1026, 1}, + {1026, 3}, + {1026, 5}, + {1026, 2}, + {1186, 0}, + {1186, 1}, {1185, 1}, - {1184, 1}, - {1184, 2}, - {1184, 1}, - {1184, 2}, - {1187, 1}, - {1187, 3}, - {922, 3}, - {1198, 0}, - {1198, 2}, - {1150, 0}, - {1150, 1}, - {907, 3}, + {1185, 2}, + {1185, 1}, + {1185, 2}, + {1188, 1}, + {1188, 3}, + {923, 3}, + {1199, 0}, + {1199, 2}, + {1151, 0}, + {1151, 1}, + {908, 3}, {763, 0}, {763, 2}, {769, 0}, {769, 3}, {838, 0}, {838, 1}, - {856, 0}, - {856, 1}, - {858, 0}, - {858, 2}, - {857, 3}, + {857, 0}, {857, 1}, - {857, 3}, - {857, 2}, - {857, 1}, - {857, 1}, - {925, 1}, - {925, 3}, - {925, 3}, + {859, 0}, + {859, 2}, + {858, 3}, + {858, 1}, + {858, 3}, + {858, 2}, + {858, 1}, + {858, 1}, + {926, 1}, + {926, 3}, + {926, 3}, {1208, 0}, {1208, 1}, {841, 2}, {841, 2}, - {886, 1}, - {886, 1}, - {886, 1}, + {887, 1}, + {887, 1}, + {887, 1}, {839, 1}, {839, 1}, {648, 1}, @@ -4675,43 +4675,43 @@ var ( {649, 1}, {649, 1}, {649, 1}, - {977, 2}, + {978, 2}, {1254, 1}, {1254, 3}, {1254, 4}, {1254, 6}, - {774, 9}, - {1043, 0}, + {773, 9}, + {1044, 0}, + {1044, 1}, + {1043, 5}, + {1043, 4}, + {1043, 4}, + {1043, 4}, + {1043, 4}, + {1043, 2}, {1043, 1}, - {1042, 5}, - {1042, 4}, - {1042, 4}, - {1042, 4}, - {1042, 4}, - {1042, 2}, - {1042, 1}, - {1042, 1}, - {1042, 1}, - {1042, 1}, - {1042, 2}, - {957, 1}, - {957, 1}, - {955, 1}, - {955, 3}, + {1043, 1}, + {1043, 1}, + {1043, 1}, + {1043, 2}, + {958, 1}, + {958, 1}, + {956, 1}, + {956, 3}, {826, 3}, {1305, 0}, {1305, 1}, {1304, 3}, {1304, 1}, - {791, 1}, - {791, 1}, - {985, 3}, - {1167, 0}, - {1167, 1}, - {1167, 3}, + {790, 1}, + {790, 1}, + {986, 3}, + {1168, 0}, + {1168, 1}, + {1168, 3}, {1232, 0}, {1232, 5}, - {776, 6}, + {775, 6}, {699, 1}, {699, 1}, {699, 1}, @@ -4726,19 +4726,19 @@ var ( {699, 2}, {700, 1}, {700, 2}, - {1143, 1}, - {1143, 3}, - {965, 2}, + {1144, 1}, + {1144, 3}, + {966, 2}, {753, 3}, - {880, 1}, - {880, 3}, + {881, 1}, + {881, 3}, {851, 1}, {851, 2}, {1243, 1}, {1243, 1}, - {929, 0}, - {929, 1}, - {929, 1}, + {930, 0}, + {930, 1}, + {930, 1}, {797, 0}, {797, 1}, {716, 3}, @@ -4792,12 +4792,12 @@ var ( {711, 3}, {768, 1}, {768, 1}, - {772, 1}, - {772, 1}, + {771, 1}, + {771, 1}, {802, 0}, {802, 1}, - {916, 0}, - {916, 1}, + {917, 0}, + {917, 1}, {800, 1}, {800, 2}, {705, 1}, @@ -4829,8 +4829,8 @@ var ( {705, 1}, {705, 1}, {705, 1}, - {1069, 0}, - {1069, 2}, + {1070, 0}, + {1070, 2}, {709, 1}, {709, 1}, {709, 1}, @@ -4877,10 +4877,10 @@ var ( {704, 7}, {704, 1}, {704, 8}, - {1196, 1}, - {1196, 1}, - {1196, 1}, - {1196, 1}, + {1197, 1}, + {1197, 1}, + {1197, 1}, + {1197, 1}, {706, 1}, {706, 1}, {707, 1}, @@ -4922,9 +4922,9 @@ var ( {1238, 2}, {702, 4}, {702, 6}, - {1195, 0}, - {1195, 2}, - {1195, 3}, + {1196, 0}, + {1196, 2}, + {1196, 3}, {816, 1}, {816, 1}, {816, 1}, @@ -4937,82 +4937,82 @@ var ( {816, 1}, {816, 1}, {816, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {793, 1}, - {1182, 0}, - {1182, 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}, + {1183, 0}, + {1183, 1}, {1314, 1}, {1314, 2}, - {1135, 4}, - {1179, 0}, - {1179, 2}, - {978, 2}, - {978, 3}, - {978, 1}, - {978, 1}, - {978, 2}, - {978, 2}, - {978, 2}, - {978, 2}, - {978, 2}, - {978, 1}, - {978, 1}, - {978, 2}, - {978, 1}, + {1136, 4}, + {1180, 0}, + {1180, 2}, + {979, 2}, + {979, 3}, + {979, 1}, + {979, 1}, + {979, 2}, + {979, 2}, + {979, 2}, + {979, 2}, + {979, 2}, + {979, 1}, + {979, 1}, + {979, 2}, + {979, 1}, {824, 1}, {824, 1}, {824, 1}, - {865, 0}, - {865, 1}, + {866, 0}, + {866, 1}, {724, 1}, {724, 3}, - {783, 1}, - {783, 3}, - {901, 2}, - {901, 4}, - {947, 1}, - {947, 3}, - {889, 0}, - {889, 2}, - {1085, 0}, - {1085, 1}, - {1082, 4}, + {782, 1}, + {782, 3}, + {902, 2}, + {902, 4}, + {948, 1}, + {948, 3}, + {890, 0}, + {890, 2}, + {1086, 0}, + {1086, 1}, + {1083, 4}, {1253, 1}, {1253, 1}, - {1022, 2}, - {1022, 4}, + {1023, 2}, + {1023, 4}, {1302, 1}, {1302, 3}, - {1001, 3}, - {1002, 1}, - {1002, 1}, - {894, 1}, - {894, 2}, - {986, 4}, - {986, 4}, - {986, 5}, - {986, 2}, - {986, 3}, - {986, 1}, - {986, 2}, - {1108, 1}, - {1092, 1}, - {1037, 2}, + {1002, 3}, + {1003, 1}, + {1003, 1}, + {895, 1}, + {895, 2}, + {987, 4}, + {987, 4}, + {987, 5}, + {987, 2}, + {987, 3}, + {987, 1}, + {987, 2}, + {1109, 1}, + {1093, 1}, + {1038, 2}, {739, 3}, {740, 3}, {741, 7}, @@ -5026,8 +5026,8 @@ var ( {1295, 0}, {1295, 1}, {1295, 1}, - {1091, 0}, - {1091, 4}, + {1092, 0}, + {1092, 4}, {738, 7}, {738, 6}, {738, 5}, @@ -5037,17 +5037,17 @@ var ( {750, 2}, {752, 2}, {752, 3}, - {1140, 3}, - {1140, 1}, - {913, 4}, - {1193, 2}, + {1141, 3}, + {1141, 1}, + {914, 4}, + {1194, 2}, {1315, 0}, {1315, 2}, {1316, 1}, {1316, 3}, - {1136, 3}, - {906, 1}, - {1138, 3}, + {1137, 3}, + {907, 1}, + {1139, 3}, {1321, 4}, {1236, 0}, {1236, 1}, @@ -5062,17 +5062,17 @@ var ( {1319, 1}, {1318, 1}, {1318, 1}, - {959, 2}, - {959, 2}, - {959, 2}, - {959, 4}, - {959, 2}, + {960, 2}, + {960, 2}, + {960, 2}, + {960, 4}, + {960, 2}, {1317, 4}, - {1137, 1}, - {1137, 2}, - {1137, 2}, - {1137, 2}, - {1137, 4}, + {1138, 1}, + {1138, 2}, + {1138, 2}, + {1138, 2}, + {1138, 4}, {749, 0}, {749, 1}, {734, 2}, @@ -5089,119 +5089,119 @@ var ( {715, 6}, {715, 6}, {715, 9}, - {1071, 0}, - {1071, 3}, - {1071, 3}, {1072, 0}, - {1072, 2}, - {863, 0}, - {863, 2}, - {863, 2}, + {1072, 3}, + {1072, 3}, + {1073, 0}, + {1073, 2}, + {864, 0}, + {864, 2}, + {864, 2}, {1237, 0}, {1237, 2}, {1237, 2}, {1292, 1}, - {868, 1}, - {868, 3}, + {869, 1}, + {869, 3}, {834, 1}, {834, 4}, - {788, 1}, - {788, 1}, - {787, 6}, - {787, 2}, - {787, 3}, + {787, 1}, + {787, 1}, + {786, 6}, + {786, 2}, + {786, 3}, {842, 0}, {842, 4}, - {900, 0}, + {901, 0}, + {901, 1}, {900, 1}, - {899, 1}, - {899, 2}, - {924, 2}, - {924, 2}, - {924, 2}, + {900, 2}, + {925, 2}, + {925, 2}, + {925, 2}, {1205, 0}, {1205, 2}, {1205, 3}, {1205, 3}, - {923, 5}, + {924, 5}, {840, 0}, {840, 1}, {840, 3}, {840, 1}, {840, 3}, - {1039, 1}, - {1039, 2}, - {1040, 0}, {1040, 1}, - {784, 3}, - {784, 5}, - {784, 7}, - {784, 7}, - {784, 9}, - {784, 4}, - {784, 6}, - {784, 3}, - {784, 5}, + {1040, 2}, + {1041, 0}, + {1041, 1}, + {783, 3}, + {783, 5}, + {783, 7}, + {783, 7}, + {783, 9}, + {783, 4}, + {783, 6}, + {783, 3}, + {783, 5}, {803, 1}, {803, 1}, - {1073, 0}, - {1073, 1}, + {1074, 0}, + {1074, 1}, {809, 1}, {809, 2}, {809, 2}, - {1048, 0}, - {1048, 2}, - {860, 1}, - {860, 1}, + {1049, 0}, + {1049, 2}, + {861, 1}, + {861, 1}, {1260, 1}, {1260, 1}, - {1188, 1}, - {1188, 1}, - {1183, 0}, - {1183, 1}, + {1189, 1}, + {1189, 1}, + {1184, 0}, + {1184, 1}, {754, 2}, {754, 4}, {754, 4}, {754, 5}, {815, 0}, {815, 1}, - {1099, 1}, - {1099, 1}, - {1099, 1}, - {1099, 1}, - {1099, 1}, - {1099, 1}, - {1099, 1}, - {1099, 1}, - {1099, 1}, + {1100, 1}, + {1100, 1}, + {1100, 1}, + {1100, 1}, + {1100, 1}, + {1100, 1}, + {1100, 1}, + {1100, 1}, + {1100, 1}, {1262, 0}, {1262, 1}, {1263, 2}, {1263, 1}, {847, 1}, - {902, 0}, - {902, 1}, - {1100, 1}, - {1100, 1}, + {903, 0}, + {903, 1}, + {1101, 1}, + {1101, 1}, {1261, 1}, - {945, 0}, - {945, 1}, - {867, 0}, - {867, 5}, + {946, 0}, + {946, 1}, + {868, 0}, + {868, 5}, {696, 3}, {696, 3}, {696, 3}, - {866, 0}, - {866, 3}, - {866, 3}, - {866, 4}, - {866, 5}, - {866, 4}, - {866, 5}, - {866, 5}, - {866, 4}, - {1062, 0}, - {1062, 2}, + {867, 0}, + {867, 3}, + {867, 3}, + {867, 4}, + {867, 5}, + {867, 4}, + {867, 5}, + {867, 5}, + {867, 4}, + {1063, 0}, + {1063, 2}, {751, 1}, {751, 1}, {751, 2}, @@ -5221,31 +5221,31 @@ var ( {1265, 2}, {1265, 2}, {1265, 2}, - {946, 1}, - {979, 9}, - {979, 9}, - {897, 2}, - {897, 4}, - {897, 6}, - {897, 4}, - {897, 4}, - {897, 3}, - {897, 6}, - {897, 6}, - {1103, 3}, - {1102, 6}, - {1101, 1}, - {1101, 1}, - {1101, 1}, + {947, 1}, + {980, 9}, + {980, 9}, + {898, 2}, + {898, 4}, + {898, 6}, + {898, 4}, + {898, 4}, + {898, 3}, + {898, 6}, + {898, 6}, + {1104, 3}, + {1103, 6}, + {1102, 1}, + {1102, 1}, + {1102, 1}, {1266, 3}, {1266, 1}, {1266, 1}, - {951, 1}, - {951, 3}, - {904, 3}, - {904, 2}, - {904, 2}, - {904, 3}, + {952, 1}, + {952, 3}, + {905, 3}, + {905, 2}, + {905, 2}, + {905, 3}, {1212, 2}, {1212, 2}, {1212, 2}, @@ -5257,28 +5257,28 @@ var ( {810, 1}, {817, 1}, {817, 3}, - {883, 1}, - {883, 3}, - {883, 3}, - {958, 3}, - {958, 4}, - {958, 4}, - {958, 4}, - {958, 3}, - {958, 3}, - {958, 2}, - {958, 4}, - {958, 4}, - {958, 2}, - {958, 2}, - {1159, 1}, - {1159, 1}, - {794, 1}, - {794, 1}, + {884, 1}, + {884, 3}, + {884, 3}, + {959, 3}, + {959, 4}, + {959, 4}, + {959, 4}, + {959, 3}, + {959, 3}, + {959, 2}, + {959, 4}, + {959, 4}, + {959, 2}, + {959, 2}, + {1160, 1}, + {1160, 1}, + {793, 1}, + {793, 1}, {852, 1}, {852, 1}, - {1134, 1}, - {1134, 3}, + {1135, 1}, + {1135, 3}, {714, 1}, {714, 1}, {713, 1}, @@ -5289,75 +5289,75 @@ var ( {760, 2}, {848, 1}, {848, 3}, - {1077, 1}, - {1077, 4}, - {871, 1}, + {1078, 1}, + {1078, 4}, + {872, 1}, {807, 1}, {807, 1}, - {786, 3}, - {786, 2}, - {943, 1}, - {943, 1}, + {785, 3}, + {785, 2}, + {944, 1}, + {944, 1}, {806, 1}, {806, 1}, {844, 1}, {844, 3}, - {960, 3}, - {960, 5}, - {960, 6}, - {960, 4}, - {960, 4}, - {960, 5}, - {960, 5}, - {960, 5}, - {960, 6}, - {960, 4}, - {960, 5}, - {960, 6}, - {960, 4}, - {960, 3}, - {960, 3}, - {960, 4}, - {960, 4}, - {960, 5}, - {960, 5}, - {960, 3}, - {960, 3}, - {960, 3}, - {960, 3}, - {960, 3}, - {960, 3}, - {960, 3}, - {960, 3}, - {1142, 2}, - {1142, 2}, - {1142, 3}, - {1142, 3}, - {1197, 1}, - {1197, 3}, - {1035, 5}, - {1059, 1}, - {1059, 3}, - {1106, 3}, - {1106, 4}, - {1106, 4}, - {1106, 5}, - {1106, 4}, - {1106, 5}, - {1106, 4}, - {1106, 4}, - {1106, 6}, - {1106, 4}, - {1106, 8}, - {1106, 2}, - {1106, 5}, - {1106, 3}, - {1106, 3}, - {1106, 2}, - {1106, 5}, - {1106, 2}, - {1106, 2}, - {1106, 4}, + {961, 3}, + {961, 5}, + {961, 6}, + {961, 4}, + {961, 4}, + {961, 5}, + {961, 5}, + {961, 5}, + {961, 6}, + {961, 4}, + {961, 5}, + {961, 6}, + {961, 4}, + {961, 3}, + {961, 3}, + {961, 4}, + {961, 4}, + {961, 5}, + {961, 5}, + {961, 3}, + {961, 3}, + {961, 3}, + {961, 3}, + {961, 3}, + {961, 3}, + {961, 3}, + {961, 3}, + {1143, 2}, + {1143, 2}, + {1143, 3}, + {1143, 3}, + {1198, 1}, + {1198, 3}, + {1036, 5}, + {1060, 1}, + {1060, 3}, + {1107, 3}, + {1107, 4}, + {1107, 4}, + {1107, 5}, + {1107, 4}, + {1107, 5}, + {1107, 4}, + {1107, 4}, + {1107, 6}, + {1107, 4}, + {1107, 8}, + {1107, 2}, + {1107, 5}, + {1107, 3}, + {1107, 3}, + {1107, 2}, + {1107, 5}, + {1107, 2}, + {1107, 2}, + {1107, 4}, {1269, 2}, {1269, 2}, {1269, 4}, @@ -5365,15 +5365,15 @@ var ( {1272, 1}, {1271, 1}, {1271, 3}, - {1105, 1}, - {1105, 1}, - {1105, 2}, - {1105, 2}, - {1105, 2}, - {1105, 1}, - {1105, 1}, - {1105, 1}, - {1105, 1}, + {1106, 1}, + {1106, 1}, + {1106, 2}, + {1106, 2}, + {1106, 2}, + {1106, 1}, + {1106, 1}, + {1106, 1}, + {1106, 1}, {1270, 0}, {1270, 3}, {1303, 0}, @@ -5381,8 +5381,8 @@ var ( {1267, 1}, {1267, 1}, {1267, 1}, - {792, 1}, - {792, 1}, + {791, 1}, + {791, 1}, {1273, 1}, {1273, 1}, {1273, 1}, @@ -5423,24 +5423,24 @@ var ( {1268, 0}, {1268, 2}, {1268, 2}, - {921, 0}, - {921, 1}, - {921, 1}, - {1067, 0}, - {1067, 1}, + {922, 0}, + {922, 1}, + {922, 1}, + {1068, 0}, + {1068, 1}, {827, 0}, {827, 2}, - {1107, 2}, - {1029, 3}, - {935, 1}, - {935, 3}, - {1192, 1}, - {1192, 1}, - {1192, 3}, - {1192, 1}, - {1192, 2}, - {1192, 3}, - {1192, 1}, + {1108, 2}, + {1030, 3}, + {936, 1}, + {936, 3}, + {1193, 1}, + {1193, 1}, + {1193, 3}, + {1193, 1}, + {1193, 2}, + {1193, 3}, + {1193, 1}, {1223, 0}, {1223, 1}, {1223, 1}, @@ -5450,110 +5450,110 @@ var ( {823, 0}, {823, 1}, {823, 1}, - {1122, 0}, - {1122, 1}, - {949, 0}, - {949, 2}, + {1123, 0}, + {1123, 1}, + {950, 0}, + {950, 2}, {1322, 0}, {1322, 3}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1112, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, - {1127, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1113, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, + {1128, 1}, {835, 1}, {835, 1}, {835, 1}, @@ -5565,13 +5565,13 @@ var ( {835, 1}, {1280, 1}, {1280, 3}, - {884, 2}, - {980, 1}, - {980, 1}, - {948, 1}, - {948, 1}, - {1120, 1}, - {1120, 3}, + {885, 2}, + {981, 1}, + {981, 1}, + {949, 1}, + {949, 1}, + {1121, 1}, + {1121, 3}, {1290, 0}, {1290, 3}, {828, 1}, @@ -5610,16 +5610,16 @@ var ( {828, 3}, {821, 0}, {821, 1}, - {1114, 1}, - {1114, 1}, - {997, 0}, - {997, 1}, - {903, 1}, - {903, 2}, - {903, 3}, + {1115, 1}, + {1115, 1}, + {998, 0}, + {998, 1}, + {904, 1}, + {904, 2}, + {904, 3}, {1240, 0}, {1240, 1}, - {1129, 3}, + {1130, 3}, {825, 3}, {825, 3}, {825, 3}, @@ -5653,19 +5653,19 @@ var ( {1211, 1}, {1211, 1}, {1211, 1}, - {1157, 1}, - {1157, 1}, - {1068, 0}, - {1068, 1}, - {1068, 1}, - {1189, 1}, - {1189, 1}, - {1189, 1}, - {1191, 1}, - {1191, 1}, - {1191, 1}, - {1191, 2}, - {1155, 1}, + {1158, 1}, + {1158, 1}, + {1069, 0}, + {1069, 1}, + {1069, 1}, + {1190, 1}, + {1190, 1}, + {1190, 1}, + {1192, 1}, + {1192, 1}, + {1192, 1}, + {1192, 2}, + {1156, 1}, {1285, 3}, {1285, 2}, {1285, 3}, @@ -5681,15 +5681,15 @@ var ( {1285, 1}, {1285, 3}, {1285, 2}, - {881, 1}, - {881, 1}, + {882, 1}, + {882, 1}, {1228, 1}, {1228, 2}, {1228, 2}, - {1133, 2}, - {1133, 2}, - {1133, 1}, - {1133, 1}, + {1134, 2}, + {1134, 2}, + {1134, 1}, + {1134, 1}, {1230, 2}, {1230, 2}, {1230, 1}, @@ -5700,24 +5700,24 @@ var ( {1230, 2}, {1325, 1}, {1325, 1}, - {1156, 1}, - {1156, 2}, - {1156, 1}, - {1156, 1}, - {1156, 2}, + {1157, 1}, + {1157, 2}, + {1157, 1}, + {1157, 1}, + {1157, 2}, {1297, 1}, {1297, 2}, {1297, 1}, {1297, 1}, - {862, 1}, - {862, 1}, - {862, 1}, - {862, 1}, - {1175, 1}, - {1175, 2}, - {1175, 2}, - {1175, 2}, - {1175, 3}, + {863, 1}, + {863, 1}, + {863, 1}, + {863, 1}, + {1176, 1}, + {1176, 2}, + {1176, 2}, + {1176, 2}, + {1176, 3}, {747, 3}, {770, 0}, {770, 1}, @@ -5726,198 +5726,198 @@ var ( {854, 1}, {855, 0}, {855, 2}, - {885, 0}, - {885, 1}, - {885, 1}, - {891, 5}, + {886, 0}, + {886, 1}, + {886, 1}, + {892, 5}, {1233, 0}, {1233, 1}, - {785, 0}, - {785, 2}, - {785, 3}, + {784, 0}, + {784, 2}, + {784, 3}, {1234, 0}, {1234, 2}, {759, 2}, {759, 1}, {759, 2}, - {1066, 0}, - {1066, 2}, + {1067, 0}, + {1067, 2}, {1283, 1}, {1283, 3}, - {950, 1}, - {950, 1}, - {950, 1}, - {1126, 1}, - {1126, 3}, + {951, 1}, + {951, 1}, + {951, 1}, + {1127, 1}, + {1127, 3}, {725, 1}, {725, 1}, {1284, 1}, {1284, 1}, {1284, 1}, - {777, 1}, - {777, 2}, + {776, 1}, + {776, 2}, {766, 10}, {766, 8}, - {1132, 2}, - {778, 2}, - {779, 0}, - {779, 1}, + {1133, 2}, + {777, 2}, + {778, 0}, + {778, 1}, {1330, 0}, {1330, 1}, - {998, 7}, - {994, 4}, - {970, 7}, - {970, 9}, - {964, 3}, + {999, 7}, + {995, 4}, + {971, 7}, + {971, 9}, + {965, 3}, {1210, 2}, {1210, 6}, - {869, 2}, - {905, 1}, - {905, 3}, - {988, 0}, + {870, 2}, + {906, 1}, + {906, 3}, + {989, 0}, + {989, 2}, + {1170, 1}, + {1170, 2}, {988, 2}, - {1169, 1}, - {1169, 2}, - {987, 2}, - {987, 2}, - {987, 2}, - {987, 2}, - {941, 0}, - {941, 1}, - {940, 2}, - {940, 2}, - {940, 2}, - {940, 2}, + {988, 2}, + {988, 2}, + {988, 2}, + {942, 0}, + {942, 1}, + {941, 2}, + {941, 2}, + {941, 2}, + {941, 2}, {1258, 1}, {1258, 3}, {1258, 2}, - {942, 2}, - {942, 2}, - {942, 2}, - {942, 2}, - {1079, 0}, + {943, 2}, + {943, 2}, + {943, 2}, + {943, 2}, + {1080, 0}, + {1080, 1}, {1079, 1}, - {1078, 1}, - {1078, 2}, - {934, 2}, - {934, 2}, - {934, 1}, - {934, 4}, - {934, 2}, - {934, 2}, - {933, 3}, - {1161, 0}, - {1152, 0}, - {1152, 3}, - {1152, 3}, - {1152, 5}, - {1152, 5}, - {1152, 4}, - {1153, 1}, - {1036, 1}, - {1036, 1}, - {1098, 1}, + {1079, 2}, + {935, 2}, + {935, 2}, + {935, 1}, + {935, 4}, + {935, 2}, + {935, 2}, + {934, 3}, + {1162, 0}, + {1153, 0}, + {1153, 3}, + {1153, 3}, + {1153, 5}, + {1153, 5}, + {1153, 4}, + {1154, 1}, + {1037, 1}, + {1037, 1}, + {1099, 1}, {1259, 1}, {1259, 3}, - {873, 1}, - {873, 1}, - {873, 1}, - {873, 1}, - {873, 1}, - {873, 1}, - {873, 1}, - {873, 1}, - {989, 7}, - {1005, 5}, - {1005, 7}, - {1034, 9}, - {1032, 7}, - {1033, 4}, - {1139, 0}, - {1139, 3}, - {1139, 3}, - {1139, 3}, - {1139, 3}, - {1139, 3}, - {919, 1}, - {919, 2}, - {944, 1}, - {944, 1}, - {944, 1}, - {944, 3}, - {944, 3}, - {1097, 1}, - {1097, 3}, - {937, 1}, - {937, 4}, - {938, 1}, - {938, 2}, - {938, 1}, - {938, 1}, - {938, 2}, - {938, 2}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 2}, - {938, 1}, - {938, 2}, - {938, 1}, - {938, 2}, - {938, 2}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 1}, - {938, 3}, - {938, 2}, - {938, 2}, - {938, 2}, - {938, 2}, - {938, 2}, - {938, 2}, - {938, 2}, - {938, 1}, + {874, 1}, + {874, 1}, + {874, 1}, + {874, 1}, + {874, 1}, + {874, 1}, + {874, 1}, + {874, 1}, + {990, 7}, + {1006, 5}, + {1006, 7}, + {1035, 9}, + {1033, 7}, + {1034, 4}, + {1140, 0}, + {1140, 3}, + {1140, 3}, + {1140, 3}, + {1140, 3}, + {1140, 3}, + {920, 1}, + {920, 2}, + {945, 1}, + {945, 1}, + {945, 1}, + {945, 3}, + {945, 3}, + {1098, 1}, + {1098, 3}, {938, 1}, - {1060, 0}, - {1060, 1}, - {1060, 1}, - {1060, 1}, - {1083, 1}, - {1083, 3}, - {1083, 3}, - {1083, 3}, - {1083, 1}, - {1096, 7}, - {1095, 4}, - {887, 15}, + {938, 4}, + {939, 1}, + {939, 2}, + {939, 1}, + {939, 1}, + {939, 2}, + {939, 2}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 2}, + {939, 1}, + {939, 2}, + {939, 1}, + {939, 2}, + {939, 2}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 1}, + {939, 3}, + {939, 2}, + {939, 2}, + {939, 2}, + {939, 2}, + {939, 2}, + {939, 2}, + {939, 2}, + {939, 1}, + {939, 1}, + {1061, 0}, + {1061, 1}, + {1061, 1}, + {1061, 1}, + {1084, 1}, + {1084, 3}, + {1084, 3}, + {1084, 3}, + {1084, 1}, + {1097, 7}, + {1096, 4}, + {888, 15}, {1203, 0}, {1203, 3}, - {1160, 0}, - {1160, 3}, - {1053, 0}, - {1053, 1}, - {1027, 0}, - {1027, 2}, + {1161, 0}, + {1161, 3}, + {1054, 0}, + {1054, 1}, + {1028, 0}, + {1028, 2}, {820, 1}, {820, 1}, - {1186, 2}, - {1186, 1}, - {1026, 3}, - {1026, 4}, - {1026, 3}, - {1026, 3}, + {1187, 2}, + {1187, 1}, + {1027, 3}, + {1027, 4}, + {1027, 3}, + {1027, 3}, {836, 1}, {836, 1}, {836, 1}, - {927, 0}, - {927, 3}, + {928, 0}, + {928, 3}, {1278, 0}, {1278, 3}, {1218, 0}, @@ -5926,60 +5926,60 @@ var ( {1220, 2}, {1219, 3}, {1219, 1}, - {1051, 3}, - {1130, 2}, - {1054, 3}, - {1124, 1}, - {1124, 1}, - {1121, 2}, + {1052, 3}, + {1131, 2}, + {1055, 3}, + {1125, 1}, + {1125, 1}, + {1122, 2}, {1222, 1}, {1222, 2}, {1222, 1}, {1222, 2}, {1291, 1}, {1291, 3}, + {1048, 2}, + {1048, 3}, + {1048, 3}, + {1047, 1}, {1047, 2}, - {1047, 3}, - {1047, 3}, - {1046, 1}, - {1046, 2}, - {1052, 3}, - {1009, 5}, - {993, 6}, - {966, 6}, - {995, 6}, - {1171, 0}, - {1171, 1}, + {1053, 3}, + {1010, 5}, + {994, 6}, + {967, 6}, + {996, 6}, + {1172, 0}, + {1172, 1}, {1264, 1}, {1264, 2}, - {896, 3}, - {896, 3}, - {896, 3}, - {896, 3}, - {896, 3}, - {896, 1}, - {896, 2}, - {896, 3}, - {896, 1}, - {896, 2}, - {896, 3}, - {896, 1}, - {896, 2}, - {896, 1}, - {896, 1}, - {896, 2}, + {897, 3}, + {897, 3}, + {897, 3}, + {897, 3}, + {897, 3}, + {897, 1}, + {897, 2}, + {897, 3}, + {897, 1}, + {897, 2}, + {897, 3}, + {897, 1}, + {897, 2}, + {897, 1}, + {897, 1}, + {897, 2}, {799, 1}, {799, 2}, {799, 2}, - {1011, 4}, - {968, 5}, - {1144, 1}, - {1144, 2}, - {967, 1}, - {967, 1}, - {967, 3}, - {967, 3}, - {1038, 8}, + {1012, 4}, + {969, 5}, + {1145, 1}, + {1145, 2}, + {968, 1}, + {968, 1}, + {968, 3}, + {968, 3}, + {1039, 8}, {1227, 0}, {1227, 2}, {1226, 0}, @@ -5988,22 +5988,22 @@ var ( {1251, 2}, {1250, 0}, {1250, 2}, - {1019, 1}, - {956, 1}, - {956, 3}, - {895, 2}, - {1081, 5}, - {1081, 6}, - {1081, 9}, - {1081, 10}, - {1081, 4}, + {1020, 1}, + {957, 1}, + {957, 3}, + {896, 2}, + {1082, 5}, + {1082, 6}, + {1082, 9}, + {1082, 10}, + {1082, 4}, } yyXErrors = map[yyXError]string{} yyParseTab = [4155][]uint16{ // 0 - {1985, 1985, 59: 2477, 80: 2592, 82: 2458, 91: 2488, 159: 2460, 163: 2482, 165: 2486, 168: 2457, 196: 2507, 205: 2453, 214: 2506, 2473, 2459, 231: 2485, 236: 2463, 239: 2483, 241: 2454, 243: 2489, 259: 2604, 261: 2475, 265: 2474, 272: 2487, 274: 2455, 277: 2476, 288: 2468, 459: 2497, 2496, 483: 2600, 2495, 492: 2481, 498: 2505, 511: 2595, 515: 2471, 553: 2494, 555: 2480, 632: 2490, 635: 2603, 640: 2456, 2594, 653: 2451, 656: 2462, 661: 2461, 666: 2504, 673: 2452, 696: 2501, 731: 2464, 738: 2503, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2574, 2573, 2467, 766: 2593, 2465, 774: 2557, 776: 2568, 2584, 790: 2466, 795: 2523, 808: 2511, 814: 2598, 837: 2596, 846: 2478, 872: 2518, 882: 2521, 887: 2560, 894: 2565, 897: 2575, 914: 2530, 918: 2469, 953: 2599, 960: 2509, 962: 2510, 2513, 2514, 966: 2516, 968: 2515, 970: 2512, 972: 2517, 2519, 2520, 976: 2479, 2556, 979: 2526, 989: 2534, 2527, 2528, 2529, 2535, 2533, 2536, 2537, 998: 2532, 2531, 1001: 2522, 2484, 2470, 2538, 2550, 2539, 2540, 2541, 2543, 2547, 2544, 2548, 2549, 2542, 2546, 2545, 1018: 2508, 1022: 2524, 2525, 2472, 1028: 2552, 2551, 1032: 2554, 2555, 2553, 1037: 2590, 2558, 1045: 2602, 2601, 2559, 1052: 2561, 1054: 2587, 1081: 2562, 2563, 1084: 2564, 1086: 2569, 1089: 2566, 2567, 1092: 2589, 2570, 2597, 2572, 2571, 1102: 2577, 2576, 2580, 1106: 2581, 1108: 2588, 1111: 2578, 2591, 1116: 2579, 1128: 2582, 2583, 2586, 1132: 2585, 1277: 2449, 1280: 2450}, + {1985, 1985, 59: 2477, 80: 2592, 82: 2458, 91: 2488, 159: 2460, 163: 2482, 165: 2486, 168: 2457, 196: 2507, 205: 2453, 214: 2506, 2473, 2459, 231: 2485, 236: 2463, 239: 2483, 241: 2454, 243: 2489, 259: 2604, 261: 2475, 265: 2474, 272: 2487, 274: 2455, 277: 2476, 288: 2468, 459: 2497, 2496, 483: 2600, 2495, 492: 2481, 498: 2505, 511: 2595, 515: 2471, 553: 2494, 555: 2480, 632: 2490, 635: 2603, 640: 2456, 2594, 653: 2451, 656: 2462, 661: 2461, 666: 2504, 673: 2452, 696: 2501, 731: 2464, 738: 2503, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2574, 2573, 2467, 766: 2593, 2465, 773: 2557, 775: 2568, 2584, 789: 2466, 795: 2523, 808: 2511, 814: 2598, 837: 2596, 846: 2478, 873: 2518, 883: 2521, 888: 2560, 895: 2565, 898: 2575, 915: 2530, 919: 2469, 954: 2599, 961: 2509, 963: 2510, 2513, 2514, 967: 2516, 969: 2515, 971: 2512, 973: 2517, 2519, 2520, 977: 2479, 2556, 980: 2526, 990: 2534, 2527, 2528, 2529, 2535, 2533, 2536, 2537, 999: 2532, 2531, 1002: 2522, 2484, 2470, 2538, 2550, 2539, 2540, 2541, 2543, 2547, 2544, 2548, 2549, 2542, 2546, 2545, 1019: 2508, 1023: 2524, 2525, 2472, 1029: 2552, 2551, 1033: 2554, 2555, 2553, 1038: 2590, 2558, 1046: 2602, 2601, 2559, 1053: 2561, 1055: 2587, 1082: 2562, 2563, 1085: 2564, 1087: 2569, 1090: 2566, 2567, 1093: 2589, 2570, 2597, 2572, 2571, 1103: 2577, 2576, 2580, 1107: 2581, 1109: 2588, 1112: 2578, 2591, 1117: 2579, 1129: 2582, 2583, 2586, 1133: 2585, 1277: 2449, 1280: 2450}, {2448}, {2447, 6601}, {26: 6542, 132: 6539, 158: 6540, 185: 6543, 331: 6541, 474: 4071, 553: 1803, 568: 5903, 833: 6538, 838: 4070}, @@ -6017,50 +6017,50 @@ var ( // 10 {386: 6433}, {461: 6432}, - {2236, 2236, 81: 5746, 491: 5744, 843: 5745, 986: 6431}, - {26: 6250, 92: 2035, 99: 2035, 132: 6246, 139: 2035, 151: 574, 156: 5401, 158: 6247, 160: 6168, 164: 6248, 185: 6251, 208: 5872, 6238, 494: 6245, 553: 2004, 568: 5903, 629: 6240, 635: 2129, 655: 2035, 663: 6242, 833: 6243, 921: 6249, 930: 5400, 1206: 6239, 1244: 6244, 1274: 6241}, - {26: 6175, 99: 6169, 110: 2004, 132: 6173, 151: 574, 156: 5401, 158: 6170, 160: 6168, 163: 997, 6171, 185: 6176, 208: 5872, 6164, 275: 6172, 553: 2004, 568: 5903, 635: 6166, 833: 6165, 921: 6174, 930: 6167}, + {2236, 2236, 81: 5746, 491: 5744, 843: 5745, 987: 6431}, + {26: 6250, 92: 2035, 99: 2035, 132: 6246, 139: 2035, 151: 574, 156: 5401, 158: 6247, 160: 6168, 164: 6248, 185: 6251, 208: 5872, 6238, 494: 6245, 553: 2004, 568: 5903, 629: 6240, 635: 2129, 655: 2035, 663: 6242, 833: 6243, 922: 6249, 931: 5400, 1206: 6239, 1244: 6244, 1274: 6241}, + {26: 6175, 99: 6169, 110: 2004, 132: 6173, 151: 574, 156: 5401, 158: 6170, 160: 6168, 163: 997, 6171, 185: 6176, 208: 5872, 6164, 275: 6172, 553: 2004, 568: 5903, 635: 6166, 833: 6165, 922: 6174, 931: 6167}, // 15 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3474, 761: 6163}, - {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 474: 818, 487: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 902: 6129}, + {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 474: 818, 487: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 903: 6129}, {2012, 2012}, {2011, 2011}, - {459: 2497, 484: 2495, 553: 2494, 632: 2490, 641: 2594, 696: 3772, 731: 2464, 738: 3771, 2491, 2492, 2493, 2502, 744: 2500, 3773, 3774, 766: 6128, 6126, 790: 6127}, + {459: 2497, 484: 2495, 553: 2494, 632: 2490, 641: 2594, 696: 3772, 731: 2464, 738: 3771, 2491, 2492, 2493, 2502, 744: 2500, 3773, 3774, 766: 6128, 6126, 789: 6127}, // 20 - {82: 2458, 159: 2460, 165: 2486, 168: 2457, 324: 6107, 459: 2497, 2496, 484: 2495, 492: 2481, 498: 6110, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6108, 731: 2464, 738: 6109, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6116, 6115, 2467, 766: 2593, 2465, 774: 6113, 776: 6114, 6112, 790: 2466, 795: 6111, 814: 6122, 872: 6118, 882: 6119, 887: 6117, 894: 6120, 897: 6121, 1127: 6106}, + {82: 2458, 159: 2460, 165: 2486, 168: 2457, 324: 6107, 459: 2497, 2496, 484: 2495, 492: 2481, 498: 6110, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6108, 731: 2464, 738: 6109, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6116, 6115, 2467, 766: 2593, 2465, 773: 6113, 775: 6114, 6112, 789: 2466, 795: 6111, 814: 6122, 873: 6118, 883: 6119, 888: 6117, 895: 6120, 898: 6121, 1128: 6106}, {2: 1982, 1982, 1982, 1982, 1982, 8: 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 58: 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 459: 1982, 1982, 479: 1982, 484: 1982, 492: 1982, 553: 1982, 555: 1982, 632: 1982, 640: 1982, 1982, 653: 1982, 731: 1982}, {2: 1981, 1981, 1981, 1981, 1981, 8: 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 58: 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 1981, 459: 1981, 1981, 479: 1981, 484: 1981, 492: 1981, 553: 1981, 555: 1981, 632: 1981, 640: 1981, 1981, 653: 1981, 731: 1981}, {2: 1980, 1980, 1980, 1980, 1980, 8: 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 58: 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 1980, 459: 1980, 1980, 479: 1980, 484: 1980, 492: 1980, 553: 1980, 555: 1980, 632: 1980, 640: 1980, 1980, 653: 1980, 731: 1980}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 6083, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 2497, 2496, 479: 6082, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 640: 6084, 2594, 648: 3805, 2664, 2665, 2663, 653: 2610, 696: 2611, 724: 6080, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 774: 2615, 776: 2616, 2614, 790: 2466, 795: 2613, 808: 2619, 835: 6081}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 6083, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 2497, 2496, 479: 6082, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 640: 6084, 2594, 648: 3805, 2664, 2665, 2663, 653: 2610, 696: 2611, 724: 6080, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 6081}, // 25 - {553: 5998, 568: 5903, 833: 5997, 975: 6076}, - {553: 5998, 568: 5903, 833: 5997, 975: 5996}, + {553: 5998, 568: 5903, 833: 5997, 976: 6076}, + {553: 5998, 568: 5903, 833: 5997, 976: 5996}, {132: 5994}, {132: 5989}, {132: 5983}, // 30 - {13: 3720, 26: 5838, 39: 5864, 5863, 98: 571, 107: 571, 110: 571, 125: 574, 132: 5827, 138: 574, 160: 5871, 180: 5836, 189: 574, 197: 5873, 5850, 203: 5859, 571, 208: 5872, 237: 5856, 260: 5855, 294: 5868, 299: 5837, 306: 5852, 5866, 309: 5844, 316: 5842, 318: 5858, 322: 5848, 325: 5857, 5831, 328: 5870, 330: 5840, 340: 5832, 348: 5846, 358: 5835, 5834, 366: 5869, 371: 5865, 5862, 5861, 387: 5853, 391: 5849, 486: 3721, 553: 5830, 634: 3719, 5839, 640: 5867, 661: 5829, 759: 5845, 898: 5860, 921: 5851, 926: 5841, 939: 5854, 1000: 5843, 1067: 5833, 1267: 5847, 1273: 5828}, + {13: 3720, 26: 5838, 39: 5864, 5863, 98: 571, 107: 571, 110: 571, 125: 574, 132: 5827, 138: 574, 160: 5871, 180: 5836, 189: 574, 197: 5873, 5850, 203: 5859, 571, 208: 5872, 237: 5856, 260: 5855, 294: 5868, 299: 5837, 306: 5852, 5866, 309: 5844, 316: 5842, 318: 5858, 322: 5848, 325: 5857, 5831, 328: 5870, 330: 5840, 340: 5832, 348: 5846, 358: 5835, 5834, 366: 5869, 371: 5865, 5862, 5861, 387: 5853, 391: 5849, 486: 3721, 553: 5830, 634: 3719, 5839, 640: 5867, 661: 5829, 759: 5845, 899: 5860, 922: 5851, 927: 5841, 940: 5854, 1001: 5843, 1068: 5833, 1267: 5847, 1273: 5828}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5816, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5818, 2664, 2665, 2663, 1254: 5817}, - {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 474: 818, 481: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 902: 5803}, - {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 481: 1020, 735: 5218, 5217, 5216, 824: 5219, 865: 5769}, + {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 474: 818, 481: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 903: 5803}, + {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 481: 1020, 735: 5218, 5217, 5216, 824: 5219, 866: 5769}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5764, 2664, 2665, 2663}, // 35 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5758, 2664, 2665, 2663}, {163: 5756}, {163: 998}, - {996, 996, 81: 5746, 491: 5744, 843: 5745, 986: 5743}, + {996, 996, 81: 5746, 491: 5744, 843: 5745, 987: 5743}, {987, 987}, // 40 {986, 986}, {461: 5742}, - {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, 823, 823, 823, 823, 823, 823, 823, 58: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 5713, 5719, 5720, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 459: 823, 461: 823, 823, 823, 823, 469: 823, 823, 823, 823, 823, 478: 823, 484: 823, 486: 823, 492: 823, 823, 500: 5716, 509: 823, 529: 823, 552: 823, 554: 823, 823, 823, 823, 823, 823, 823, 823, 823, 564: 823, 823, 823, 823, 823, 823, 572: 823, 574: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 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, 638: 3432, 732: 3430, 3431, 735: 5218, 5217, 5216, 743: 5213, 755: 5712, 5715, 5711, 768: 5634, 772: 5709, 824: 5710, 847: 5708, 1099: 5718, 5714, 1262: 5707, 5717}, - {237, 237, 57: 237, 458: 237, 460: 237, 466: 237, 468: 237, 476: 237, 237, 479: 237, 237, 237, 483: 237, 487: 5682, 237, 2624, 237, 499: 237, 778: 2625, 5683, 1193: 5681}, - {813, 813, 57: 813, 458: 813, 460: 813, 466: 813, 468: 813, 476: 813, 813, 479: 813, 813, 813, 483: 813, 488: 813, 490: 813, 499: 5672, 922: 5674, 945: 5673}, + {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, 823, 823, 823, 823, 823, 823, 823, 58: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 5713, 5719, 5720, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 459: 823, 461: 823, 823, 823, 823, 469: 823, 823, 823, 823, 823, 478: 823, 484: 823, 486: 823, 492: 823, 823, 500: 5716, 509: 823, 529: 823, 552: 823, 554: 823, 823, 823, 823, 823, 823, 823, 823, 823, 564: 823, 823, 823, 823, 823, 823, 572: 823, 574: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 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, 638: 3432, 732: 3430, 3431, 735: 5218, 5217, 5216, 743: 5213, 755: 5712, 5715, 5711, 768: 5634, 771: 5709, 824: 5710, 847: 5708, 1100: 5718, 5714, 1262: 5707, 5717}, + {237, 237, 57: 237, 458: 237, 460: 237, 466: 237, 468: 237, 476: 237, 237, 479: 237, 237, 237, 483: 237, 487: 5682, 237, 2624, 237, 499: 237, 777: 2625, 5683, 1194: 5681}, + {813, 813, 57: 813, 458: 813, 460: 813, 466: 813, 468: 813, 476: 813, 813, 479: 813, 813, 813, 483: 813, 488: 813, 490: 813, 499: 5672, 923: 5674, 946: 5673}, // 45 {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 5668}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5663}, - {561: 3780, 895: 3779, 956: 3778}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5650, 2664, 2665, 2663, 913: 5649, 1140: 5647, 1255: 5648}, + {561: 3780, 896: 3779, 957: 3778}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5650, 2664, 2665, 2663, 914: 5649, 1141: 5647, 1255: 5648}, {459: 2497, 2496, 484: 2495, 553: 2494, 632: 2490, 696: 5646, 738: 3765, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 3767, 3766, 3764}, // 50 {795, 795, 57: 795, 458: 795, 460: 795, 468: 795}, @@ -6071,7 +6071,7 @@ var ( // 55 {475, 475, 466: 781, 476: 781, 781}, {237: 5615, 260: 5614}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 5498, 5503, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 5501, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 5500, 3252, 2738, 2742, 5504, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 5505, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5499, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 5506, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 5502, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 464: 5508, 486: 3721, 554: 5512, 574: 5511, 634: 3719, 648: 5509, 2664, 2665, 2663, 759: 5513, 817: 5510, 958: 5514, 1134: 5507}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 5498, 5503, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 5501, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 5500, 3252, 2738, 2742, 5504, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 5505, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5499, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 5506, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 5502, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 464: 5508, 486: 3721, 554: 5512, 574: 5511, 634: 3719, 648: 5509, 2664, 2665, 2663, 759: 5513, 817: 5510, 959: 5514, 1135: 5507}, {27: 5383, 196: 5388, 203: 5386, 205: 5381, 5387, 264: 5385, 300: 5384, 5389, 304: 5382, 319: 5390, 365: 5391, 571: 5380, 846: 5379}, {31: 550, 110: 550, 125: 550, 136: 4621, 142: 550, 180: 550, 186: 550, 195: 550, 211: 550, 222: 550, 242: 550, 245: 550, 529: 550, 553: 550, 804: 4620, 823: 5352}, // 60 @@ -6178,25 +6178,25 @@ var ( {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, 380, 380, 380, 380, 380, 380, 380, 58: 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 553: 5349, 1240: 5350}, // 145 {243, 243, 468: 243}, - {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 459: 818, 474: 818, 565: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 902: 5215}, + {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 459: 818, 474: 818, 565: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 903: 5215}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5211, 2664, 2665, 2663, 801: 5212}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5056, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 5058, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 5064, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 5060, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5057, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 5065, 3095, 2831, 3050, 5059, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 5062, 5166, 2745, 2981, 5063, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 5061, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5067, 483: 5090, 555: 5084, 631: 5088, 5073, 635: 5083, 638: 5077, 641: 5086, 648: 3377, 2664, 2665, 2663, 653: 5078, 656: 5082, 661: 5079, 725: 5066, 731: 5081, 786: 5068, 814: 5072, 837: 5087, 846: 5085, 919: 5069, 937: 5070, 5076, 943: 5071, 5074, 952: 5080, 954: 5089, 1097: 5167}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5056, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 5058, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 5064, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 5060, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5057, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 5065, 3095, 2831, 3050, 5059, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 5062, 2744, 2745, 2981, 5063, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 5061, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5067, 483: 5090, 555: 5084, 631: 5088, 5073, 635: 5083, 638: 5077, 641: 5086, 648: 3377, 2664, 2665, 2663, 653: 5078, 656: 5082, 661: 5079, 725: 5066, 731: 5081, 786: 5068, 814: 5072, 837: 5087, 846: 5085, 919: 5069, 937: 5070, 5076, 943: 5071, 5074, 952: 5080, 954: 5089, 1097: 5075}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5056, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 5058, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 5064, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 5060, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5057, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 5065, 3095, 2831, 3050, 5059, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 5062, 5166, 2745, 2981, 5063, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 5061, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5067, 483: 5090, 555: 5084, 631: 5088, 5073, 635: 5083, 638: 5077, 641: 5086, 648: 3377, 2664, 2665, 2663, 653: 5078, 656: 5082, 661: 5079, 725: 5066, 731: 5081, 785: 5068, 814: 5072, 837: 5087, 846: 5085, 920: 5069, 938: 5070, 5076, 944: 5071, 5074, 953: 5080, 955: 5089, 1098: 5167}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5056, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 5058, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 5064, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 5060, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5057, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 5065, 3095, 2831, 3050, 5059, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 5062, 2744, 2745, 2981, 5063, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 5061, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5067, 483: 5090, 555: 5084, 631: 5088, 5073, 635: 5083, 638: 5077, 641: 5086, 648: 3377, 2664, 2665, 2663, 653: 5078, 656: 5082, 661: 5079, 725: 5066, 731: 5081, 785: 5068, 814: 5072, 837: 5087, 846: 5085, 920: 5069, 938: 5070, 5076, 944: 5071, 5074, 953: 5080, 955: 5089, 1098: 5075}, // 150 {32: 5015, 275: 5016}, - {110: 5002, 553: 5003, 1124: 5014}, - {110: 5002, 553: 5003, 1124: 5001}, + {110: 5002, 553: 5003, 1125: 5014}, + {110: 5002, 553: 5003, 1125: 5001}, {37: 4997, 143: 4998, 493: 2638, 722: 4996}, {37: 56, 143: 56, 211: 4995, 493: 56}, // 155 {290: 4978}, {363: 2605}, {315: 2606, 814: 2607}, - {918: 2609}, + {919: 2609}, {461: 2608}, // 160 {1, 1}, - {186: 2622, 459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 640: 2621, 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 774: 2615, 776: 2616, 2614, 790: 2466, 795: 2613, 808: 2619, 835: 2620}, + {186: 2622, 459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 640: 2621, 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 2620}, {474: 4071, 553: 1803, 838: 4070}, {436, 436, 466: 780, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, {438, 438, 466: 781, 476: 781, 781}, @@ -6210,10 +6210,10 @@ var ( {437, 437}, {435, 435}, {5, 5}, - {186: 4065, 459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 774: 2615, 776: 2616, 2614, 790: 2466, 795: 2613, 808: 2619, 835: 4064}, + {186: 4065, 459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 4064}, {143: 2623}, // 175 - {237, 237, 480: 237, 488: 237, 2624, 237, 778: 2625, 2626}, + {237, 237, 480: 237, 488: 237, 2624, 237, 777: 2625, 2626}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 4063}, {236, 236, 57: 236, 458: 236, 460: 236, 466: 236, 468: 236, 476: 236, 236, 479: 236, 236, 236, 483: 236, 488: 236, 490: 236, 499: 236, 501: 236, 236}, {1258, 1258, 480: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 2629}, @@ -6221,14 +6221,14 @@ var ( // 180 {1257, 1257, 57: 1257, 124: 1257, 458: 1257, 460: 1257, 466: 1257, 468: 1257, 476: 1257, 1257, 479: 1257, 1257, 1257, 483: 1257, 488: 1257}, {834, 834, 480: 2630, 488: 2631, 754: 2632, 815: 2633}, - {493: 2638, 564: 2640, 722: 2637, 730: 2639, 860: 2647}, - {8: 2634, 254: 2635, 1188: 2636}, + {493: 2638, 564: 2640, 722: 2637, 730: 2639, 861: 2647}, + {8: 2634, 254: 2635, 1189: 2636}, {833, 833, 57: 833, 458: 833, 460: 833, 466: 833, 468: 833, 476: 833, 833, 479: 833, 481: 833, 483: 833}, // 185 {3, 3}, {493: 842, 510: 842, 561: 842, 564: 842}, {493: 841, 510: 841, 561: 841, 564: 841}, - {493: 2638, 510: 840, 561: 840, 564: 2640, 722: 2637, 730: 2639, 860: 2641, 1183: 2642}, + {493: 2638, 510: 840, 561: 840, 564: 2640, 722: 2637, 730: 2639, 861: 2641, 1184: 2642}, {1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 13: 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 59: 1922, 61: 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 92: 1922, 1922, 1922, 1922, 1922, 1922, 100: 1922, 103: 1922, 105: 1922, 1922, 108: 1922, 1922, 111: 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 1922, 162: 1922, 199: 1922, 1922, 458: 1922, 1922, 1922, 464: 1922, 1922, 1922, 1922, 1922, 474: 1922, 1922, 1922, 1922, 479: 1922, 481: 1922, 483: 1922, 1922, 1922, 1922, 492: 1922, 510: 1922, 553: 1922, 561: 1922, 632: 1922, 634: 1922, 1922, 640: 1922}, // 190 {1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 13: 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 61: 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 100: 1920, 103: 1920, 105: 1920, 1920, 108: 1920, 1920, 111: 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 1920, 126: 1920, 1920, 1920, 1920, 162: 1920, 175: 1920, 179: 1920, 199: 1920, 1920, 458: 1920, 1920, 1920, 464: 1920, 1920, 1920, 1920, 1920, 474: 1920, 1920, 1920, 1920, 479: 1920, 1920, 1920, 483: 1920, 1920, 1920, 1920, 488: 1920, 1920, 492: 1920, 510: 1920, 553: 1920, 561: 1920, 632: 1920, 634: 1920, 1920, 640: 1920, 644: 1920, 1920}, @@ -6243,11 +6243,11 @@ var ( {835, 835, 57: 835, 458: 835, 460: 835, 466: 835, 468: 835, 476: 835, 835, 479: 835, 481: 835, 483: 835}, {838, 838, 7: 2648, 57: 838, 162: 2649, 458: 838, 460: 838, 466: 838, 468: 838, 476: 838, 838, 479: 838, 481: 838, 483: 838}, // 200 - {493: 2638, 564: 2640, 722: 2637, 730: 2639, 860: 2651}, - {493: 2638, 564: 2640, 722: 2637, 730: 2639, 860: 2650}, + {493: 2638, 564: 2640, 722: 2637, 730: 2639, 861: 2651}, + {493: 2638, 564: 2640, 722: 2637, 730: 2639, 861: 2650}, {836, 836, 57: 836, 458: 836, 460: 836, 466: 836, 468: 836, 476: 836, 836, 479: 836, 481: 836, 483: 836}, {837, 837, 57: 837, 458: 837, 460: 837, 466: 837, 468: 837, 476: 837, 837, 479: 837, 481: 837, 483: 837}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 880: 3128}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 881: 3128}, // 205 {1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 4060, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 460: 1486, 1486, 1486, 1486, 465: 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 476: 1486, 1486, 479: 1486, 1486, 1486, 1486, 1486, 485: 1486, 487: 1486, 1486, 1486, 1486, 1486, 494: 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 530: 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 570: 1486, 639: 1486, 642: 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, 4057, 1485, 1485, 1485, 1485, 465: 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 476: 1485, 1485, 479: 1485, 1485, 1485, 1485, 1485, 485: 1485, 487: 1485, 1485, 1485, 1485, 1485, 494: 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 530: 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 570: 1485, 639: 1485, 642: 1485, 1485}, @@ -6256,9 +6256,9 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 4051}, // 210 {459: 4023}, - {1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 460: 1882, 1882, 465: 1882, 1882, 468: 1882, 1882, 1882, 474: 1882, 476: 1882, 1882, 479: 1882, 1882, 1882, 4006, 1882, 485: 1882, 487: 1882, 1882, 1882, 1882, 1882, 494: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 508: 1882, 510: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 531: 1882, 1882, 4003, 4001, 4000, 4008, 4002, 4004, 4005, 4007, 1168: 3999, 1213: 3998}, + {1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 460: 1882, 1882, 465: 1882, 1882, 468: 1882, 1882, 1882, 474: 1882, 476: 1882, 1882, 479: 1882, 1882, 1882, 4006, 1882, 485: 1882, 487: 1882, 1882, 1882, 1882, 1882, 494: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 508: 1882, 510: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 531: 1882, 1882, 4003, 4001, 4000, 4008, 4002, 4004, 4005, 4007, 1169: 3999, 1213: 3998}, {1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 460: 1857, 1857, 465: 1857, 1857, 468: 1857, 1857, 1857, 474: 1857, 476: 1857, 1857, 479: 1857, 1857, 1857, 1857, 1857, 485: 1857, 487: 1857, 1857, 1857, 1857, 1857, 494: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 508: 1857, 510: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 531: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857}, - {1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 460: 1830, 1830, 3970, 3969, 465: 1830, 1830, 468: 1830, 1830, 1830, 3573, 3572, 3578, 1830, 476: 1830, 1830, 479: 1830, 1830, 1830, 1830, 1830, 485: 1830, 487: 1830, 1830, 1830, 1830, 1830, 494: 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 3974, 1830, 3574, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 530: 3973, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 3971, 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571, 3980, 3981, 796: 3972, 1088: 3975, 1154: 3977, 1209: 3976, 1216: 3978, 1256: 3979}, + {1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 460: 1830, 1830, 3970, 3969, 465: 1830, 1830, 468: 1830, 1830, 1830, 3573, 3572, 3578, 1830, 476: 1830, 1830, 479: 1830, 1830, 1830, 1830, 1830, 485: 1830, 487: 1830, 1830, 1830, 1830, 1830, 494: 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 3974, 1830, 3574, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 530: 3973, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 3971, 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571, 3980, 3981, 796: 3972, 1089: 3975, 1155: 3977, 1209: 3976, 1216: 3978, 1256: 3979}, {1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 3966, 1779, 1779, 1779, 1779, 465: 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 476: 1779, 1779, 479: 1779, 1779, 1779, 1779, 1779, 485: 1779, 487: 1779, 1779, 1779, 1779, 1779, 494: 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 530: 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 570: 1779, 639: 1779, 642: 1779, 1779}, // 215 {1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 652: 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778}, @@ -6849,7 +6849,7 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3750, 3140, 3223, 3139, 3136}, {459: 3745}, // 705 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 528: 1043, 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3732, 1182: 3733}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 528: 1043, 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3732, 1183: 3733}, {459: 3674}, {459: 3671}, {459: 3663}, @@ -6877,10 +6877,10 @@ var ( {1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 465: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 476: 1157, 1157, 479: 1157, 1157, 1157, 1157, 1157, 485: 1157, 487: 1157, 1157, 1157, 1157, 1157, 494: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 530: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 570: 1157}, {459: 3660}, {459: 3657}, - {1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 3654, 1168, 1168, 1168, 1168, 465: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 476: 1168, 1168, 479: 1168, 1168, 1168, 1168, 1168, 485: 1168, 487: 1168, 1168, 1168, 1168, 1168, 494: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 530: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 570: 1168, 1069: 3655}, + {1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 3654, 1168, 1168, 1168, 1168, 465: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 476: 1168, 1168, 479: 1168, 1168, 1168, 1168, 1168, 485: 1168, 487: 1168, 1168, 1168, 1168, 1168, 494: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 530: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 570: 1168, 1070: 3655}, // 730 {459: 3652}, - {1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 3648, 1075, 1075, 1075, 1075, 465: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 476: 1075, 1075, 479: 1075, 1075, 1075, 1075, 1075, 485: 1075, 487: 1075, 1075, 1075, 1075, 1075, 494: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 530: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 570: 1075, 1195: 3647}, + {1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 3648, 1075, 1075, 1075, 1075, 465: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 476: 1075, 1075, 479: 1075, 1075, 1075, 1075, 1075, 485: 1075, 487: 1075, 1075, 1075, 1075, 1075, 494: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 530: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 570: 1075, 1196: 3647}, {459: 3639}, {459: 3635}, {459: 3630}, @@ -6954,7 +6954,7 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3375, 648: 3377, 2664, 2665, 2663, 725: 3374, 852: 3373}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3372, 3140, 3223, 3139, 3136}, {144: 900, 474: 900, 487: 3242, 727: 900, 1237: 3241}, - {144: 3246, 474: 3247, 727: 903, 863: 3245}, + {144: 3246, 474: 3247, 727: 903, 864: 3245}, {8: 3243, 336: 3244}, // 795 {144: 899, 474: 899, 727: 899}, @@ -6965,7 +6965,7 @@ var ( // 800 {727: 901}, {727: 902}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 3287, 648: 3286, 2664, 2665, 2663, 906: 3289, 1138: 3290, 1320: 3288}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 3287, 648: 3286, 2664, 2665, 2663, 907: 3289, 1139: 3290, 1320: 3288}, {909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 460: 909, 909, 909, 909, 465: 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 476: 909, 909, 479: 909, 909, 909, 909, 909, 485: 909, 487: 909, 909, 909, 909, 909, 494: 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 530: 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 570: 909}, {1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 652: 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779}, // 805 @@ -7009,7 +7009,7 @@ var ( {1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 652: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 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, 652: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339}, {951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 460: 951, 951, 951, 951, 465: 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 479: 951, 951, 951, 951, 951, 485: 951, 487: 951, 951, 951, 951, 951, 494: 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 530: 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 570: 951}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 948, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 475: 948, 490: 948, 510: 948, 513: 948, 948, 648: 3286, 2664, 2665, 2663, 906: 3293, 1236: 3292, 1321: 3291}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 948, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 475: 948, 490: 948, 510: 948, 513: 948, 948, 648: 3286, 2664, 2665, 2663, 907: 3293, 1236: 3292, 1321: 3291}, // 840 {922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 460: 922, 922, 922, 922, 465: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 476: 922, 922, 479: 922, 922, 922, 922, 922, 485: 922, 487: 922, 922, 922, 922, 922, 494: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 530: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 570: 922}, {921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 460: 921, 921, 921, 921, 465: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 476: 921, 921, 479: 921, 921, 921, 921, 921, 485: 921, 487: 921, 921, 921, 921, 921, 494: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 530: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 570: 921}, @@ -7020,18 +7020,18 @@ var ( {57: 947, 475: 947, 490: 947, 510: 947, 513: 947, 947}, {57: 944, 490: 3301, 510: 944, 513: 944, 944, 1242: 3300}, {646: 3296}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 880: 3297}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 881: 3297}, {7: 3298, 57: 945, 490: 945, 510: 945, 513: 945, 945}, // 850 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3299}, {1266, 1266, 7: 1266, 57: 1266, 124: 1266, 458: 1266, 460: 1266, 466: 1266, 468: 1266, 476: 1266, 1266, 479: 1266, 1266, 1266, 483: 1266, 488: 1266, 490: 1266, 501: 1266, 1266, 510: 1266, 513: 1266, 1266}, {57: 942, 510: 3306, 513: 3307, 3308, 1241: 3304, 1319: 3305}, {646: 3302}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 880: 3303}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 881: 3303}, // 855 {7: 3298, 57: 943, 510: 943, 513: 943, 943}, {57: 949}, - {145: 3319, 157: 3315, 493: 3309, 541: 3320, 559: 3311, 3310, 564: 3317, 567: 3318, 805: 3316, 959: 3313, 1317: 3314, 3312}, + {145: 3319, 157: 3315, 493: 3309, 541: 3320, 559: 3311, 3310, 564: 3317, 567: 3318, 805: 3316, 960: 3313, 1317: 3314, 3312}, {145: 940, 157: 940, 493: 940, 541: 940, 559: 940, 940, 564: 940, 567: 940}, {145: 939, 157: 939, 493: 939, 541: 939, 559: 939, 939, 564: 939, 567: 939}, // 860 @@ -7049,7 +7049,7 @@ var ( // 870 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3369}, {561: 3368}, - {145: 3319, 157: 3321, 493: 3309, 559: 3311, 3310, 564: 3323, 567: 3324, 805: 3322, 959: 3326, 1137: 3325}, + {145: 3319, 157: 3321, 493: 3309, 559: 3311, 3310, 564: 3323, 567: 3324, 805: 3322, 960: 3326, 1138: 3325}, {130: 3366, 147: 3367}, {130: 3364, 147: 3365}, // 875 @@ -7057,10 +7057,10 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3329}, {491: 3327}, {57: 929, 491: 929}, - {145: 3319, 157: 3321, 493: 3309, 559: 3311, 3310, 564: 3323, 567: 3324, 805: 3322, 959: 3326, 1137: 3328}, + {145: 3319, 157: 3321, 493: 3309, 559: 3311, 3310, 564: 3323, 567: 3324, 805: 3322, 960: 3326, 1138: 3328}, // 880 {57: 930}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 793: 3330, 816: 3359}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 792: 3330, 816: 3359}, {1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 465: 1072, 1072, 468: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 476: 1072, 1072, 479: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 487: 1072, 1072, 1072, 1072, 1072, 1072, 494: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 530: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 553: 1072, 632: 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, 465: 1071, 1071, 468: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 476: 1071, 1071, 479: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 487: 1071, 1071, 1071, 1071, 1071, 1071, 494: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 530: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 553: 1071, 632: 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, 465: 1070, 1070, 468: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 476: 1070, 1070, 479: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 487: 1070, 1070, 1070, 1070, 1070, 1070, 494: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 530: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 553: 1070, 632: 1070}, @@ -7108,7 +7108,7 @@ var ( {57: 928, 491: 928}, // 920 {57: 931, 491: 931}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 793: 3330, 816: 3370}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 792: 3330, 816: 3370}, {130: 3360}, {950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 460: 950, 950, 950, 950, 465: 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 476: 950, 950, 479: 950, 950, 950, 950, 950, 485: 950, 487: 950, 950, 950, 950, 950, 494: 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 530: 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 570: 950}, {1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 460: 1224, 1224, 1224, 1224, 465: 1224, 1224, 3238, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 476: 1224, 1224, 479: 1224, 1224, 1224, 1224, 1224, 485: 1224, 487: 1224, 1224, 1224, 1224, 1224, 494: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 530: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 570: 1224}, @@ -7125,35 +7125,35 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3382}, {57: 3383, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, // 935 - {144: 3246, 474: 3247, 727: 903, 863: 3384}, + {144: 3246, 474: 3247, 727: 903, 864: 3384}, {727: 3250, 734: 3385}, {910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 460: 910, 910, 910, 910, 465: 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 476: 910, 910, 479: 910, 910, 910, 910, 910, 485: 910, 487: 910, 910, 910, 910, 910, 494: 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 530: 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 570: 910}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3387}, {57: 3388, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, // 940 - {144: 3246, 474: 3247, 727: 903, 863: 3389}, + {144: 3246, 474: 3247, 727: 903, 864: 3389}, {727: 3250, 734: 3390}, {911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 460: 911, 911, 911, 911, 465: 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 476: 911, 911, 479: 911, 911, 911, 911, 911, 485: 911, 487: 911, 911, 911, 911, 911, 494: 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 530: 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 570: 911}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3392}, - {7: 3394, 57: 908, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228, 1071: 3393}, + {7: 3394, 57: 908, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228, 1072: 3393}, // 945 {57: 3401}, {493: 3309, 559: 3311, 3310, 564: 3396, 805: 3395}, - {7: 3398, 57: 905, 1072: 3400}, - {7: 3398, 57: 905, 1072: 3397}, + {7: 3398, 57: 905, 1073: 3400}, + {7: 3398, 57: 905, 1073: 3397}, {57: 906}, // 950 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3399}, {57: 904, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {57: 907}, - {144: 3246, 474: 3247, 727: 903, 863: 3402}, + {144: 3246, 474: 3247, 727: 903, 864: 3402}, {727: 3250, 734: 3403}, // 955 {912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 460: 912, 912, 912, 912, 465: 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 476: 912, 912, 479: 912, 912, 912, 912, 912, 485: 912, 487: 912, 912, 912, 912, 912, 494: 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 530: 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 570: 912}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3405}, - {7: 3394, 57: 908, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228, 1071: 3406}, + {7: 3394, 57: 908, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228, 1072: 3406}, {57: 3407}, - {144: 3246, 474: 3247, 727: 903, 863: 3408}, + {144: 3246, 474: 3247, 727: 903, 864: 3408}, // 960 {727: 3250, 734: 3409}, {913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 460: 913, 913, 913, 913, 465: 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 476: 913, 913, 479: 913, 913, 913, 913, 913, 485: 913, 487: 913, 913, 913, 913, 913, 494: 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 530: 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 570: 913}, @@ -7180,7 +7180,7 @@ var ( {727: 3250, 734: 3428}, // 980 {919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 460: 919, 919, 919, 919, 465: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 476: 919, 919, 479: 919, 919, 919, 919, 919, 485: 919, 487: 919, 919, 919, 919, 919, 494: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 530: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 570: 919}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3436, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3436, 802: 3435}, {2: 1207, 1207, 1207, 1207, 1207, 8: 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 58: 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 459: 1207, 461: 1207, 1207, 1207, 1207, 469: 1207, 1207, 1207, 1207, 1207, 478: 1207, 484: 1207, 486: 1207, 492: 1207, 1207, 500: 1207, 509: 1207, 529: 1207, 552: 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 564: 1207, 1207, 1207, 1207, 1207, 1207, 572: 1207, 574: 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 1207, 632: 1207, 636: 1207, 638: 1207, 732: 1207, 1207, 735: 1207, 1207, 1207, 743: 1207, 755: 1207, 1207, 1207}, {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, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 58: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 459: 1206, 461: 1206, 1206, 1206, 1206, 469: 1206, 1206, 1206, 1206, 1206, 478: 1206, 484: 1206, 486: 1206, 492: 1206, 1206, 500: 1206, 509: 1206, 529: 1206, 552: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 564: 1206, 1206, 1206, 1206, 1206, 1206, 572: 1206, 574: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 632: 1206, 636: 1206, 638: 1206, 732: 1206, 1206, 735: 1206, 1206, 1206, 743: 1206, 755: 1206, 1206, 1206}, {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, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 58: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 459: 1205, 461: 1205, 1205, 1205, 1205, 469: 1205, 1205, 1205, 1205, 1205, 478: 1205, 484: 1205, 486: 1205, 492: 1205, 1205, 500: 1205, 509: 1205, 529: 1205, 552: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 564: 1205, 1205, 1205, 1205, 1205, 1205, 572: 1205, 574: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 632: 1205, 636: 1205, 638: 1205, 732: 1205, 1205, 735: 1205, 1205, 1205, 743: 1205, 755: 1205, 1205, 1205}, @@ -7195,43 +7195,43 @@ var ( {1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 460: 1086, 1086, 1086, 1086, 465: 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 476: 1086, 1086, 479: 1086, 1086, 1086, 1086, 1086, 485: 1086, 487: 1086, 1086, 1086, 1086, 1086, 494: 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 530: 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 570: 1086}, {923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 460: 923, 923, 923, 923, 465: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 476: 923, 923, 479: 923, 923, 923, 923, 923, 485: 923, 487: 923, 923, 923, 923, 923, 494: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 530: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 570: 923}, {2: 1198, 1198, 1198, 1198, 1198, 8: 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 58: 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 459: 1198, 461: 1198, 1198, 1198, 1198, 469: 1198, 1198, 1198, 1198, 1198, 478: 1198, 484: 1198, 486: 1198, 492: 1198, 1198, 529: 1198, 552: 1198, 554: 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 564: 1198, 1198, 1198, 1198, 1198, 1198, 572: 1198, 574: 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 1198, 636: 1198}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3443, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3443, 802: 3435}, // 995 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3444}, {57: 3445, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 460: 924, 924, 924, 924, 465: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 476: 924, 924, 479: 924, 924, 924, 924, 924, 485: 924, 487: 924, 924, 924, 924, 924, 494: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 530: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 570: 924, 727: 3250, 734: 3440, 749: 3446}, {1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 460: 1087, 1087, 1087, 1087, 465: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 476: 1087, 1087, 479: 1087, 1087, 1087, 1087, 1087, 485: 1087, 487: 1087, 1087, 1087, 1087, 1087, 494: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 530: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 570: 1087}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3448, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3448, 802: 3435}, // 1000 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3449}, {57: 3450, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 460: 924, 924, 924, 924, 465: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 476: 924, 924, 479: 924, 924, 924, 924, 924, 485: 924, 487: 924, 924, 924, 924, 924, 494: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 530: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 570: 924, 727: 3250, 734: 3440, 749: 3451}, {1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 460: 1088, 1088, 1088, 1088, 465: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 476: 1088, 1088, 479: 1088, 1088, 1088, 1088, 1088, 485: 1088, 487: 1088, 1088, 1088, 1088, 1088, 494: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 530: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 570: 1088}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3453, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3453, 802: 3435}, // 1005 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3454}, {57: 3455, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 460: 924, 924, 924, 924, 465: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 476: 924, 924, 479: 924, 924, 924, 924, 924, 485: 924, 487: 924, 924, 924, 924, 924, 494: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 530: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 570: 924, 727: 3250, 734: 3440, 749: 3456}, {1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 460: 1089, 1089, 1089, 1089, 465: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 476: 1089, 1089, 479: 1089, 1089, 1089, 1089, 1089, 485: 1089, 487: 1089, 1089, 1089, 1089, 1089, 494: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 530: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 570: 1089}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3458, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3458, 802: 3435}, // 1010 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3459}, {57: 3460, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 460: 924, 924, 924, 924, 465: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 476: 924, 924, 479: 924, 924, 924, 924, 924, 485: 924, 487: 924, 924, 924, 924, 924, 494: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 530: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 570: 924, 727: 3250, 734: 3440, 749: 3461}, {1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 460: 1090, 1090, 1090, 1090, 465: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 476: 1090, 1090, 479: 1090, 1090, 1090, 1090, 1090, 485: 1090, 487: 1090, 1090, 1090, 1090, 1090, 494: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 530: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 570: 1090}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3463, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3463, 802: 3435}, // 1015 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3464}, {57: 3465, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 460: 924, 924, 924, 924, 465: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 476: 924, 924, 479: 924, 924, 924, 924, 924, 485: 924, 487: 924, 924, 924, 924, 924, 494: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 530: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 570: 924, 727: 3250, 734: 3440, 749: 3466}, {1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 460: 1091, 1091, 1091, 1091, 465: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 476: 1091, 1091, 479: 1091, 1091, 1091, 1091, 1091, 485: 1091, 487: 1091, 1091, 1091, 1091, 1091, 494: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 530: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 570: 1091}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3468, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3468, 802: 3435}, // 1020 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3469}, {57: 3470, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 460: 924, 924, 924, 924, 465: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 476: 924, 924, 479: 924, 924, 924, 924, 924, 485: 924, 487: 924, 924, 924, 924, 924, 494: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 530: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 570: 924, 727: 3250, 734: 3440, 749: 3471}, {1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 460: 1092, 1092, 1092, 1092, 465: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 476: 1092, 1092, 479: 1092, 1092, 1092, 1092, 1092, 485: 1092, 487: 1092, 1092, 1092, 1092, 1092, 494: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 530: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 570: 1092}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3473, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3473, 802: 3435}, // 1025 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3474, 761: 3475}, {1870, 1870, 7: 1870, 57: 1870, 124: 1870, 468: 1870, 490: 1870, 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, @@ -7307,7 +7307,7 @@ var ( // 1085 {57: 3534, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 460: 1121, 1121, 1121, 1121, 465: 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 476: 1121, 1121, 479: 1121, 1121, 1121, 1121, 1121, 485: 1121, 487: 1121, 1121, 1121, 1121, 1121, 494: 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 530: 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 570: 1121}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3536, 1158: 3538, 1214: 3539, 1298: 3540, 3537}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3536, 1159: 3538, 1214: 3539, 1298: 3540, 3537}, {57: 3548, 487: 3549, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 3542, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3541}, // 1090 @@ -7374,19 +7374,19 @@ var ( {1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 460: 1248, 1248, 1248, 1248, 465: 1248, 1248, 468: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 476: 1248, 1248, 479: 1248, 1248, 1248, 1248, 1248, 485: 1248, 487: 1248, 1248, 1248, 1248, 1248, 494: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 530: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 3579, 1248, 1248, 1248, 1248, 1248, 1248}, {1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 460: 1251, 1251, 1251, 1251, 465: 1251, 1251, 468: 1251, 1251, 1251, 1251, 1251, 3578, 1251, 476: 1251, 1251, 479: 1251, 1251, 1251, 1251, 1251, 485: 1251, 487: 1251, 1251, 1251, 1251, 1251, 494: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 3574, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 530: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 3575, 3576, 1251, 3579, 1251, 3577, 1251, 1251, 1251, 1251}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3591}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 793: 3330, 816: 3592}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 792: 3330, 816: 3592}, {1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 460: 1249, 1249, 1249, 1249, 465: 1249, 1249, 468: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 476: 1249, 1249, 479: 1249, 1249, 1249, 1249, 1249, 485: 1249, 487: 1249, 1249, 1249, 1249, 1249, 494: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 530: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249}, // 1145 {1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 460: 1252, 1252, 1252, 1252, 465: 1252, 1252, 468: 1252, 1252, 1252, 1252, 1252, 3578, 1252, 476: 1252, 1252, 479: 1252, 1252, 1252, 1252, 1252, 485: 1252, 487: 1252, 1252, 1252, 1252, 1252, 494: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3574, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 530: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3575, 3576, 1252, 3579, 1252, 3577, 1252, 1252, 1252, 1252}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3595}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 793: 3330, 816: 3596}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 792: 3330, 816: 3596}, {1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 460: 1250, 1250, 1250, 1250, 465: 1250, 1250, 468: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 476: 1250, 1250, 479: 1250, 1250, 1250, 1250, 1250, 485: 1250, 487: 1250, 1250, 1250, 1250, 1250, 494: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 530: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250}, {1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 460: 1253, 1253, 1253, 1253, 465: 1253, 1253, 468: 1253, 1253, 1253, 3573, 3572, 3578, 1253, 476: 1253, 1253, 479: 1253, 1253, 1253, 1253, 1253, 485: 1253, 487: 1253, 1253, 1253, 1253, 1253, 494: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3574, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 530: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3575, 3576, 1253, 3579, 1253, 3577, 1253, 1253, 1253, 1253}, // 1150 {1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 460: 1254, 1254, 1254, 1254, 465: 1254, 1254, 468: 1254, 1254, 1254, 3573, 3572, 3578, 1254, 476: 1254, 1254, 479: 1254, 1254, 1254, 1254, 1254, 485: 1254, 487: 1254, 1254, 1254, 1254, 1254, 494: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3574, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 530: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3575, 3576, 1254, 3579, 1254, 3577, 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, 460: 1255, 1255, 1255, 1255, 465: 1255, 1255, 468: 1255, 1255, 1255, 3573, 3572, 3578, 1255, 476: 1255, 1255, 479: 1255, 1255, 1255, 1255, 1255, 485: 1255, 487: 1255, 1255, 1255, 1255, 1255, 494: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 3574, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 530: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 3575, 3576, 1255, 3579, 1255, 3577, 3570, 3571, 1255, 1255}, {1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 460: 1256, 1256, 1256, 1256, 465: 1256, 1256, 468: 1256, 1256, 1256, 3573, 3572, 3578, 1256, 476: 1256, 1256, 479: 1256, 1256, 1256, 1256, 1256, 485: 1256, 487: 1256, 1256, 1256, 1256, 1256, 494: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 3574, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 530: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 3575, 3576, 3569, 3579, 1256, 3577, 3570, 3571, 1256, 1256}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 793: 3330, 816: 3602}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 792: 3330, 816: 3602}, {487: 3603}, // 1155 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3604}, @@ -7397,7 +7397,7 @@ var ( // 1160 {567: 3609}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3610}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 793: 3330, 816: 3611}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 792: 3330, 816: 3611}, {57: 3612}, {1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 460: 1139, 1139, 1139, 1139, 465: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 476: 1139, 1139, 479: 1139, 1139, 1139, 1139, 1139, 485: 1139, 487: 1139, 1139, 1139, 1139, 1139, 494: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 530: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 570: 1139}, // 1165 @@ -7407,17 +7407,17 @@ var ( {57: 3621, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3618}, // 1170 - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 793: 3330, 816: 3619}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 792: 3330, 816: 3619}, {57: 3620}, {1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 460: 1140, 1140, 1140, 1140, 465: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 476: 1140, 1140, 479: 1140, 1140, 1140, 1140, 1140, 485: 1140, 487: 1140, 1140, 1140, 1140, 1140, 494: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 530: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 570: 1140}, {1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 460: 1141, 1141, 1141, 1141, 465: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 476: 1141, 1141, 479: 1141, 1141, 1141, 1141, 1141, 485: 1141, 487: 1141, 1141, 1141, 1141, 1141, 494: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 530: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 570: 1141}, - {57: 1864, 493: 3624, 1030: 3623, 3625}, + {57: 1864, 493: 3624, 1031: 3623, 3625}, // 1175 {57: 1863}, {57: 1862}, {57: 3626}, {1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 460: 1142, 1142, 1142, 1142, 465: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 476: 1142, 1142, 479: 1142, 1142, 1142, 1142, 1142, 485: 1142, 487: 1142, 1142, 1142, 1142, 1142, 494: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 530: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 570: 1142}, - {57: 1864, 493: 3624, 1030: 3623, 3628}, + {57: 1864, 493: 3624, 1031: 3623, 3628}, // 1180 {57: 3629}, {1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 460: 1143, 1143, 1143, 1143, 465: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 476: 1143, 1143, 479: 1143, 1143, 1143, 1143, 1143, 485: 1143, 487: 1143, 1143, 1143, 1143, 1143, 494: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 530: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 570: 1143}, @@ -7435,7 +7435,7 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3474, 761: 3640}, {7: 3476, 57: 3641, 468: 3642}, {1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 460: 1151, 1151, 1151, 1151, 465: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 476: 1151, 1151, 479: 1151, 1151, 1151, 1151, 1151, 485: 1151, 487: 1151, 1151, 1151, 1151, 1151, 494: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 530: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 570: 1151}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 3643}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 3643}, // 1195 {57: 3646}, {724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 98: 724, 107: 724, 458: 724, 724, 724, 462: 724, 724, 724, 724, 724, 724, 474: 724, 724, 478: 724, 484: 724, 724, 724, 492: 724, 498: 724, 529: 724, 553: 724, 563: 724, 571: 724, 573: 724, 629: 724, 724, 724, 724, 724, 724, 724, 645: 724}, @@ -7476,8 +7476,8 @@ var ( {1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 460: 1211, 1211, 1211, 1211, 465: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 476: 1211, 1211, 479: 1211, 1211, 1211, 1211, 1211, 485: 1211, 487: 1211, 1211, 1211, 1211, 1211, 494: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 530: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 570: 1211}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3675}, {7: 3676, 468: 3677, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {60: 3688, 105: 3684, 170: 3685, 3683, 174: 3690, 187: 3687, 486: 3695, 529: 3681, 634: 3694, 667: 3686, 3691, 3692, 672: 3693, 726: 3689, 881: 3682, 978: 3680}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 3678}, + {60: 3688, 105: 3684, 170: 3685, 3683, 174: 3690, 187: 3687, 486: 3695, 529: 3681, 634: 3694, 667: 3686, 3691, 3692, 672: 3693, 726: 3689, 882: 3682, 979: 3680}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 3678}, // 1230 {57: 3679}, {1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 460: 1212, 1212, 1212, 1212, 465: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 476: 1212, 1212, 479: 1212, 1212, 1212, 1212, 1212, 485: 1212, 487: 1212, 1212, 1212, 1212, 1212, 494: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 530: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 570: 1212}, @@ -7488,14 +7488,14 @@ var ( {57: 1034}, {57: 1033}, {57: 278, 459: 3710, 747: 3711, 770: 3714}, - {57: 271, 459: 3697, 747: 3698, 885: 3713, 891: 3699}, + {57: 271, 459: 3697, 747: 3698, 886: 3713, 892: 3699}, {57: 278, 459: 3710, 747: 3711, 770: 3709}, // 1240 - {57: 342, 670: 3706, 3707, 1068: 3708}, - {57: 342, 670: 3706, 3707, 1068: 3705}, + {57: 342, 670: 3706, 3707, 1069: 3708}, + {57: 342, 670: 3706, 3707, 1069: 3705}, {57: 1027}, {57: 1026}, - {57: 271, 459: 3697, 747: 3698, 885: 3696, 891: 3699}, + {57: 271, 459: 3697, 747: 3698, 886: 3696, 892: 3699}, // 1245 {57: 1024}, {13: 316, 57: 316, 459: 316, 486: 316, 529: 316, 634: 316}, @@ -7523,11 +7523,11 @@ var ( // 1265 {57: 1031}, {57: 1032}, - {13: 3720, 57: 265, 486: 3721, 529: 3717, 634: 3719, 759: 3718, 785: 3716}, + {13: 3720, 57: 265, 486: 3721, 529: 3717, 634: 3719, 759: 3718, 784: 3716}, {57: 1035}, {262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 3720, 57: 262, 458: 262, 462: 262, 262, 262, 262, 467: 262, 475: 262, 478: 262, 486: 3721, 563: 262, 571: 262, 573: 262, 629: 262, 262, 262, 633: 262, 3719, 759: 3728, 1234: 3727}, // 1270 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 3724}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 3724}, {498: 3723}, {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, 259, 259, 259, 259, 259, 259, 259, 58: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 461: 259, 464: 259, 482: 259, 489: 259, 507: 259, 529: 259}, {498: 3722}, @@ -7539,14 +7539,14 @@ var ( {263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 57: 263, 458: 263, 462: 263, 263, 263, 263, 467: 263, 475: 263, 478: 263, 563: 263, 571: 263, 573: 263, 629: 263, 263, 263, 633: 263}, {264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 57: 264, 458: 264, 462: 264, 264, 264, 264, 467: 264, 475: 264, 478: 264, 563: 264, 571: 264, 573: 264, 629: 264, 264, 264, 633: 264}, // 1280 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 3729}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 3729}, {261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 57: 261, 458: 261, 462: 261, 261, 261, 261, 467: 261, 475: 261, 478: 261, 563: 261, 571: 261, 573: 261, 629: 261, 261, 261, 633: 261}, {57: 1036}, {1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 460: 1213, 1213, 1213, 1213, 465: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 476: 1213, 1213, 479: 1213, 1213, 1213, 1213, 1213, 485: 1213, 487: 1213, 1213, 1213, 1213, 1213, 494: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 530: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 570: 1213}, {491: 3234, 494: 3232, 3233, 3231, 3229, 528: 1042, 720: 3230, 3228}, // 1285 - {528: 3736, 1135: 3735, 1314: 3734}, - {152: 1038, 528: 3736, 531: 3742, 1135: 3741, 1179: 3740}, + {528: 3736, 1136: 3735, 1314: 3734}, + {152: 1038, 528: 3736, 531: 3742, 1136: 3741, 1180: 3740}, {152: 1041, 528: 1041, 531: 1041}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3737}, {491: 3234, 494: 3232, 3233, 3231, 3229, 532: 3738, 720: 3230, 3228}, @@ -7561,7 +7561,7 @@ var ( {1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 460: 1214, 1214, 1214, 1214, 465: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 476: 1214, 1214, 479: 1214, 1214, 1214, 1214, 1214, 485: 1214, 487: 1214, 1214, 1214, 1214, 1214, 494: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 530: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 570: 1214}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3746}, {465: 3747, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {60: 3688, 105: 3684, 170: 3685, 3683, 174: 3690, 187: 3687, 486: 3695, 529: 3681, 634: 3694, 667: 3686, 3691, 3692, 672: 3693, 726: 3689, 881: 3682, 978: 3748}, + {60: 3688, 105: 3684, 170: 3685, 3683, 174: 3690, 187: 3687, 486: 3695, 529: 3681, 634: 3694, 667: 3686, 3691, 3692, 672: 3693, 726: 3689, 882: 3682, 979: 3748}, // 1300 {57: 3749}, {1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 460: 1215, 1215, 1215, 1215, 465: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 476: 1215, 1215, 479: 1215, 1215, 1215, 1215, 1215, 485: 1215, 487: 1215, 1215, 1215, 1215, 1215, 494: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 530: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 570: 1215}, @@ -7581,7 +7581,7 @@ var ( {7: 3858}, {7: 1222, 57: 1222, 462: 1222, 1222, 466: 780, 1222, 471: 1222, 1222, 1222, 476: 780, 780, 480: 2630, 482: 1222, 488: 2631, 490: 2627, 1222, 494: 1222, 1222, 1222, 1222, 507: 1222, 509: 1222, 530: 1222, 533: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 570: 1222, 753: 3775, 3776}, // 1315 - {459: 3663, 561: 3780, 895: 3779, 956: 3778}, + {459: 3663, 561: 3780, 896: 3779, 957: 3778}, {459: 2497, 484: 2495, 553: 2494, 632: 2490, 696: 3772, 738: 3771, 2491, 2492, 2493, 2502, 744: 2500, 3773, 3774}, {57: 3770, 466: 781, 476: 781, 781}, {57: 3769}, @@ -7602,7 +7602,7 @@ var ( {1258, 1258, 7: 3792, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 3791}, {8, 8, 7: 8, 57: 8, 458: 8, 460: 8, 466: 8, 468: 8, 476: 8, 8, 479: 8, 8, 8, 483: 8, 488: 8, 490: 8}, {459: 3781, 826: 3782}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 1298, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 3786, 1304: 3785, 3784}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 1298, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 3786, 1304: 3785, 3784}, {6, 6, 7: 6, 57: 6, 458: 6, 460: 6, 466: 6, 468: 6, 476: 6, 6, 479: 6, 6, 6, 483: 6, 488: 6, 490: 6}, // 1335 {1294, 1294, 7: 1294, 57: 1294, 458: 1294, 468: 1294, 480: 1294, 489: 1294, 1294, 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, @@ -7611,26 +7611,26 @@ var ( {7: 1295, 57: 1295}, {1293, 1293, 7: 1293, 57: 1293, 458: 1293, 3671, 468: 1293, 480: 1293, 489: 1293, 1293}, // 1340 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 3789}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 3789}, {7: 1296, 57: 1296}, {1299, 1299, 7: 1299, 15: 1299, 57: 1299, 458: 1299, 460: 1299, 466: 1299, 468: 1299, 476: 1299, 1299, 479: 1299, 1299, 1299, 483: 1299, 488: 1299, 490: 1299, 1299}, {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2630, 834, 483: 834, 488: 2631, 754: 2632, 815: 3794}, - {561: 3780, 895: 3793}, + {561: 3780, 896: 3793}, // 1345 {7, 7, 7: 7, 57: 7, 458: 7, 460: 7, 466: 7, 468: 7, 476: 7, 7, 479: 7, 7, 7, 483: 7, 488: 7, 490: 7}, - {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 866: 3795}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 867: 3821}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 867: 3795}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 3821}, {271: 3802, 641: 3801}, {530: 3798}, // 1350 {271: 3799}, {201: 3800}, {798, 798, 57: 798, 458: 798, 460: 798, 466: 798, 468: 798, 476: 798, 798, 481: 798}, - {797, 797, 57: 797, 137: 797, 148: 797, 167: 797, 458: 797, 460: 797, 466: 797, 468: 797, 476: 797, 797, 481: 797, 1061: 3804, 3815}, - {797, 797, 57: 797, 137: 797, 148: 797, 458: 797, 460: 797, 466: 797, 468: 797, 476: 797, 797, 481: 797, 1061: 3804, 3803}, + {797, 797, 57: 797, 137: 797, 148: 797, 167: 797, 458: 797, 460: 797, 466: 797, 468: 797, 476: 797, 797, 481: 797, 1062: 3804, 3815}, + {797, 797, 57: 797, 137: 797, 148: 797, 458: 797, 460: 797, 466: 797, 468: 797, 476: 797, 797, 481: 797, 1062: 3804, 3803}, // 1355 {804, 804, 57: 804, 137: 3813, 148: 3812, 458: 804, 460: 804, 466: 804, 468: 804, 476: 804, 804, 481: 804}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 3807}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 3807}, {1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 530: 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 553: 1018, 563: 1018, 570: 1018, 1018, 1018, 1018, 629: 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 3810, 1018, 1018, 644: 1018, 1018, 1018, 653: 1018, 656: 1018, 1018, 1018, 1018, 1018, 1018, 666: 1018, 673: 1018, 1018, 676: 1018, 691: 1018}, {1016, 1016, 7: 1016, 57: 1016, 137: 1016, 148: 1016, 167: 1016, 458: 1016, 460: 1016, 466: 1016, 468: 1016, 476: 1016, 1016, 481: 1016, 487: 1016, 637: 1016, 657: 1016, 659: 1016}, {796, 796, 7: 3808, 57: 796, 137: 796, 148: 796, 167: 796, 458: 796, 460: 796, 466: 796, 468: 796, 476: 796, 796, 481: 796}, @@ -7654,34 +7654,34 @@ var ( {1245: 3823}, // 1375 {461: 3824}, - {94, 94, 57: 94, 98: 3828, 107: 3827, 458: 94, 460: 94, 466: 94, 468: 94, 476: 94, 94, 645: 94, 820: 3826, 1027: 3825}, - {81, 81, 57: 81, 458: 81, 460: 81, 466: 81, 468: 81, 476: 81, 81, 645: 3849, 927: 3848}, - {765: 3831, 773: 3833, 781: 3834, 3832, 1026: 3830, 1186: 3829}, - {92, 92, 27: 92, 59: 92, 61: 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 458: 92, 92, 487: 92, 530: 92, 640: 92, 765: 92, 773: 92, 781: 92, 92}, + {94, 94, 57: 94, 98: 3828, 107: 3827, 458: 94, 460: 94, 466: 94, 468: 94, 476: 94, 94, 645: 94, 820: 3826, 1028: 3825}, + {81, 81, 57: 81, 458: 81, 460: 81, 466: 81, 468: 81, 476: 81, 81, 645: 3849, 928: 3848}, + {765: 3831, 772: 3833, 780: 3834, 3832, 1027: 3830, 1187: 3829}, + {92, 92, 27: 92, 59: 92, 61: 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 458: 92, 92, 487: 92, 530: 92, 640: 92, 765: 92, 772: 92, 780: 92, 92}, // 1380 - {91, 91, 27: 91, 59: 91, 61: 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 458: 91, 91, 487: 91, 530: 91, 640: 91, 765: 91, 773: 91, 781: 91, 91}, - {93, 93, 57: 93, 458: 93, 93, 93, 466: 93, 468: 93, 474: 93, 476: 93, 93, 498: 93, 645: 93, 765: 3831, 773: 3833, 781: 3834, 3832, 1026: 3847}, - {89, 89, 57: 89, 458: 89, 89, 89, 466: 89, 468: 89, 474: 89, 476: 89, 89, 498: 89, 645: 89, 765: 89, 773: 89, 781: 89, 89}, + {91, 91, 27: 91, 59: 91, 61: 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 458: 91, 91, 487: 91, 530: 91, 640: 91, 765: 91, 772: 91, 780: 91, 91}, + {93, 93, 57: 93, 458: 93, 93, 93, 466: 93, 468: 93, 474: 93, 476: 93, 93, 498: 93, 645: 93, 765: 3831, 772: 3833, 780: 3834, 3832, 1027: 3847}, + {89, 89, 57: 89, 458: 89, 89, 89, 466: 89, 468: 89, 474: 89, 476: 89, 89, 498: 89, 645: 89, 765: 89, 772: 89, 780: 89, 89}, {646: 3845}, - {773: 3842}, + {772: 3842}, // 1385 {646: 3840}, {646: 3835}, {461: 3837, 562: 3838, 566: 3839, 836: 3836}, - {85, 85, 57: 85, 458: 85, 85, 85, 466: 85, 468: 85, 474: 85, 476: 85, 85, 498: 85, 645: 85, 765: 85, 773: 85, 781: 85, 85}, - {84, 84, 57: 84, 458: 84, 84, 84, 466: 84, 468: 84, 474: 84, 476: 84, 84, 498: 84, 645: 84, 765: 84, 773: 84, 781: 84, 84}, + {85, 85, 57: 85, 458: 85, 85, 85, 466: 85, 468: 85, 474: 85, 476: 85, 85, 498: 85, 645: 85, 765: 85, 772: 85, 780: 85, 85}, + {84, 84, 57: 84, 458: 84, 84, 84, 466: 84, 468: 84, 474: 84, 476: 84, 84, 498: 84, 645: 84, 765: 84, 772: 84, 780: 84, 84}, // 1390 - {83, 83, 57: 83, 458: 83, 83, 83, 466: 83, 468: 83, 474: 83, 476: 83, 83, 498: 83, 645: 83, 765: 83, 773: 83, 781: 83, 83}, - {82, 82, 57: 82, 458: 82, 82, 82, 466: 82, 468: 82, 474: 82, 476: 82, 82, 498: 82, 645: 82, 765: 82, 773: 82, 781: 82, 82}, + {83, 83, 57: 83, 458: 83, 83, 83, 466: 83, 468: 83, 474: 83, 476: 83, 83, 498: 83, 645: 83, 765: 83, 772: 83, 780: 83, 83}, + {82, 82, 57: 82, 458: 82, 82, 82, 466: 82, 468: 82, 474: 82, 476: 82, 82, 498: 82, 645: 82, 765: 82, 772: 82, 780: 82, 82}, {461: 3837, 562: 3838, 566: 3839, 836: 3841}, - {86, 86, 57: 86, 458: 86, 86, 86, 466: 86, 468: 86, 474: 86, 476: 86, 86, 498: 86, 645: 86, 765: 86, 773: 86, 781: 86, 86}, + {86, 86, 57: 86, 458: 86, 86, 86, 466: 86, 468: 86, 474: 86, 476: 86, 86, 498: 86, 645: 86, 765: 86, 772: 86, 780: 86, 86}, {646: 3843}, // 1395 {461: 3837, 562: 3838, 566: 3839, 836: 3844}, - {87, 87, 57: 87, 458: 87, 87, 87, 466: 87, 468: 87, 474: 87, 476: 87, 87, 498: 87, 645: 87, 765: 87, 773: 87, 781: 87, 87}, + {87, 87, 57: 87, 458: 87, 87, 87, 466: 87, 468: 87, 474: 87, 476: 87, 87, 498: 87, 645: 87, 765: 87, 772: 87, 780: 87, 87}, {461: 3837, 562: 3838, 566: 3839, 836: 3846}, - {88, 88, 57: 88, 458: 88, 88, 88, 466: 88, 468: 88, 474: 88, 476: 88, 88, 498: 88, 645: 88, 765: 88, 773: 88, 781: 88, 88}, - {90, 90, 57: 90, 458: 90, 90, 90, 466: 90, 468: 90, 474: 90, 476: 90, 90, 498: 90, 645: 90, 765: 90, 773: 90, 781: 90, 90}, + {88, 88, 57: 88, 458: 88, 88, 88, 466: 88, 468: 88, 474: 88, 476: 88, 88, 498: 88, 645: 88, 765: 88, 772: 88, 780: 88, 88}, + {90, 90, 57: 90, 458: 90, 90, 90, 466: 90, 468: 90, 474: 90, 476: 90, 90, 498: 90, 645: 90, 765: 90, 772: 90, 780: 90, 90}, // 1400 {810, 810, 57: 810, 458: 810, 460: 810, 466: 810, 468: 810, 476: 810, 810}, {79, 79, 57: 79, 458: 79, 79, 79, 466: 79, 468: 79, 474: 79, 476: 79, 79, 498: 79, 765: 79, 1278: 3850, 3851}, @@ -7755,7 +7755,7 @@ var ( {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 460: 924, 924, 924, 924, 465: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 476: 924, 924, 479: 924, 924, 924, 924, 924, 485: 924, 487: 924, 924, 924, 924, 924, 494: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 530: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 570: 924, 727: 3250, 734: 3440, 749: 3907}, {1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 460: 1083, 1083, 1083, 1083, 465: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 476: 1083, 1083, 479: 1083, 1083, 1083, 1083, 1083, 485: 1083, 487: 1083, 1083, 1083, 1083, 1083, 494: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 530: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 570: 1083}, // 1460 - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 793: 3909}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 792: 3909}, {7: 3910}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3911}, {7: 3912, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, @@ -7763,7 +7763,7 @@ var ( // 1465 {57: 3914, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 460: 1130, 1130, 1130, 1130, 465: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 476: 1130, 1130, 479: 1130, 1130, 1130, 1130, 1130, 485: 1130, 487: 1130, 1130, 1130, 1130, 1130, 494: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 530: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 570: 1130}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 793: 3916}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 792: 3916}, {7: 3917}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3918}, // 1470 @@ -7771,7 +7771,7 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3920}, {57: 3921, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 460: 1131, 1131, 1131, 1131, 465: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 476: 1131, 1131, 479: 1131, 1131, 1131, 1131, 1131, 485: 1131, 487: 1131, 1131, 1131, 1131, 1131, 494: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 530: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 570: 1131}, - {170: 3925, 3924, 187: 3926, 212: 3927, 1196: 3923}, + {170: 3925, 3924, 187: 3926, 212: 3927, 1197: 3923}, // 1475 {7: 3928}, {7: 1120}, @@ -7803,7 +7803,7 @@ var ( {57: 3947, 465: 3948, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, {1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 460: 1125, 1125, 1125, 1125, 465: 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 476: 1125, 1125, 479: 1125, 1125, 1125, 1125, 1125, 485: 1125, 487: 1125, 1125, 1125, 1125, 1125, 494: 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 530: 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 570: 1125}, // 1500 - {486: 3695, 529: 3950, 634: 3694, 881: 3949}, + {486: 3695, 529: 3950, 634: 3694, 882: 3949}, {459: 3710, 747: 3953}, {459: 3710, 747: 3951}, {57: 3952}, @@ -7818,7 +7818,7 @@ var ( {57: 3959}, {1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 460: 1144, 1144, 1144, 1144, 465: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 476: 1144, 1144, 479: 1144, 1144, 1144, 1144, 1144, 485: 1144, 487: 1144, 1144, 1144, 1144, 1144, 494: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 530: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 570: 1144}, {1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 460: 1149, 1149, 1149, 1149, 465: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 476: 1149, 1149, 479: 1149, 1149, 1149, 1149, 1149, 485: 1149, 487: 1149, 1149, 1149, 1149, 1149, 494: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 530: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 570: 1149}, - {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 772: 3434, 800: 3962, 802: 3435}, + {2: 1203, 1203, 1203, 1203, 1203, 8: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 58: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 459: 1203, 461: 1203, 1203, 1203, 1203, 469: 1203, 1203, 1203, 1203, 1203, 478: 1203, 484: 1203, 486: 1203, 492: 1203, 1203, 529: 1203, 552: 1203, 554: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 564: 1203, 1203, 1203, 1203, 1203, 1203, 572: 1203, 574: 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 1203, 636: 1203, 638: 3432, 732: 3430, 3431, 768: 3433, 771: 3434, 800: 3962, 802: 3435}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3963}, // 1515 {57: 3964, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, @@ -7831,7 +7831,7 @@ var ( {146: 2232, 166: 2232, 183: 2232, 478: 2232, 507: 2232, 530: 2232, 541: 2232, 550: 2232, 2232, 557: 2232, 2232, 569: 2232}, {146: 2231, 166: 2231, 183: 2231, 478: 2231, 507: 2231, 530: 2231, 541: 2231, 550: 2231, 2231, 557: 2231, 2231, 569: 2231}, {2: 1848, 1848, 1848, 1848, 1848, 8: 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 58: 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 459: 1848, 461: 1848, 1848, 464: 1848, 469: 1848, 1848, 1848, 1848, 1848, 478: 1848, 484: 1848, 486: 1848, 492: 1848, 1848, 529: 1848, 552: 1848, 554: 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 564: 1848, 1848, 1848, 1848, 1848, 1848, 572: 1848, 574: 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848, 1848}, - {507: 3996, 530: 3995, 541: 3994, 550: 3980, 3981, 1088: 3997}, + {507: 3996, 530: 3995, 541: 3994, 550: 3980, 3981, 1089: 3997}, // 1525 {459: 1844}, {2: 1842, 1842, 1842, 1842, 1842, 8: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 58: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 459: 1842, 461: 1842, 1842, 464: 1842, 469: 1842, 1842, 1842, 1842, 1842, 478: 1842, 484: 1842, 486: 1842, 492: 1842, 1842, 529: 1842, 552: 1842, 554: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 564: 1842, 1842, 1842, 1842, 1842, 1842, 572: 1842, 574: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842}, @@ -7864,7 +7864,7 @@ var ( {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, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 58: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 459: 1839, 461: 1839, 1839, 464: 1839, 469: 1839, 1839, 1839, 1839, 1839, 478: 1839, 484: 1839, 486: 1839, 492: 1839, 1839, 529: 1839, 552: 1839, 554: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 564: 1839, 1839, 1839, 1839, 1839, 1839, 572: 1839, 574: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839}, // 1550 {166: 4021, 478: 4022, 557: 4020, 4019}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 4013, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 4014, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 4012, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 638: 4015, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 4010, 1149: 4011}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 4013, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 4014, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 4012, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 638: 4015, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 4010, 1150: 4011}, {2: 1856, 1856, 1856, 1856, 1856, 8: 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 58: 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 459: 1856, 461: 1856, 1856, 464: 1856, 469: 1856, 1856, 1856, 1856, 1856, 478: 1856, 484: 1856, 486: 1856, 492: 1856, 1856, 529: 1856, 552: 1856, 554: 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 564: 1856, 1856, 1856, 1856, 1856, 1856, 572: 1856, 574: 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 1856, 638: 1856}, {2: 1855, 1855, 1855, 1855, 1855, 8: 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 58: 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 459: 1855, 461: 1855, 1855, 464: 1855, 469: 1855, 1855, 1855, 1855, 1855, 478: 1855, 484: 1855, 486: 1855, 492: 1855, 1855, 529: 1855, 552: 1855, 554: 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 564: 1855, 1855, 1855, 1855, 1855, 1855, 572: 1855, 574: 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 1855, 638: 1855}, {2: 1854, 1854, 1854, 1854, 1854, 8: 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 58: 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 459: 1854, 461: 1854, 1854, 464: 1854, 469: 1854, 1854, 1854, 1854, 1854, 478: 1854, 484: 1854, 486: 1854, 492: 1854, 1854, 529: 1854, 552: 1854, 554: 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 564: 1854, 1854, 1854, 1854, 1854, 1854, 572: 1854, 574: 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 1854, 638: 1854}, @@ -7893,16 +7893,16 @@ var ( {1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 460: 1883, 1883, 465: 1883, 1883, 468: 1883, 1883, 1883, 474: 1883, 476: 1883, 1883, 479: 1883, 1883, 1883, 483: 1883, 485: 1883, 487: 1883, 1883, 1883, 1883, 1883, 494: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 508: 1883, 510: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 531: 1883, 1883}, {1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 460: 1861, 1861, 465: 1861, 1861, 468: 1861, 1861, 1861, 474: 1861, 476: 1861, 1861, 479: 1861, 1861, 1861, 1861, 1861, 485: 1861, 487: 1861, 1861, 1861, 1861, 1861, 494: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 508: 1861, 510: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 531: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861}, // 1575 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 4026}, - {2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 459: 2255, 2255, 475: 2255, 482: 2255, 486: 2255, 498: 2255, 515: 2255, 2255, 529: 2255, 634: 2255, 639: 4047, 656: 2255, 2255, 659: 2255, 664: 2255, 2255, 667: 2255, 2255, 2255, 2255, 2255, 2255, 675: 2255, 677: 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 692: 2255, 2255, 2255, 2255}, - {2252, 2252, 7: 2252, 57: 2252, 460: 2252}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 4026}, + {2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 459: 2255, 475: 2255, 482: 2255, 486: 2255, 498: 2255, 515: 2255, 2255, 529: 2255, 634: 2255, 639: 4047, 656: 2255, 2255, 659: 2255, 664: 2255, 2255, 667: 2255, 2255, 2255, 2255, 2255, 2255, 675: 2255, 677: 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 692: 2255, 2255, 2255, 2255}, + {7: 2252, 57: 2252}, {7: 4027, 57: 4028}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4046}, // 1580 {291: 4029}, {459: 4030}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4031}, - {57: 1879, 460: 4034, 471: 3573, 3572, 3578, 509: 3574, 530: 4033, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571, 1194: 4032}, + {57: 1879, 460: 4034, 471: 3573, 3572, 3578, 509: 3574, 530: 4033, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571, 1195: 4032}, {57: 4045}, // 1585 {218: 4038, 504: 4037}, @@ -7920,12 +7920,12 @@ var ( {240: 4044}, {57: 1877}, {1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 460: 1886, 1886, 465: 1886, 1886, 468: 1886, 1886, 1886, 474: 1886, 476: 1886, 1886, 479: 1886, 1886, 1886, 483: 1886, 485: 1886, 487: 1886, 1886, 1886, 1886, 1886, 494: 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 508: 1886, 510: 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 531: 1886, 1886}, - {2251, 2251, 7: 2251, 57: 2251, 460: 2251}, + {7: 2251, 57: 2251}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4048, 2664, 2665, 2663}, // 1600 - {2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 459: 2254, 2254, 475: 2254, 482: 2254, 486: 2254, 498: 2254, 515: 2254, 2254, 529: 2254, 634: 2254, 639: 4049, 656: 2254, 2254, 659: 2254, 664: 2254, 2254, 667: 2254, 2254, 2254, 2254, 2254, 2254, 675: 2254, 677: 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 692: 2254, 2254, 2254, 2254}, + {2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 459: 2254, 475: 2254, 482: 2254, 486: 2254, 498: 2254, 515: 2254, 2254, 529: 2254, 634: 2254, 639: 4049, 656: 2254, 2254, 659: 2254, 664: 2254, 2254, 667: 2254, 2254, 2254, 2254, 2254, 2254, 675: 2254, 677: 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 692: 2254, 2254, 2254, 2254}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4050, 2664, 2665, 2663}, - {2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 459: 2253, 2253, 475: 2253, 482: 2253, 486: 2253, 498: 2253, 515: 2253, 2253, 529: 2253, 634: 2253, 656: 2253, 2253, 659: 2253, 664: 2253, 2253, 667: 2253, 2253, 2253, 2253, 2253, 2253, 675: 2253, 677: 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 692: 2253, 2253, 2253, 2253}, + {2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 459: 2253, 475: 2253, 482: 2253, 486: 2253, 498: 2253, 515: 2253, 2253, 529: 2253, 634: 2253, 656: 2253, 2253, 659: 2253, 664: 2253, 2253, 667: 2253, 2253, 2253, 2253, 2253, 2253, 675: 2253, 677: 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 2253, 692: 2253, 2253, 2253, 2253}, {1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 460: 1887, 1887, 465: 1887, 1887, 468: 1887, 1887, 1887, 474: 1887, 476: 1887, 1887, 479: 1887, 1887, 1887, 483: 1887, 485: 1887, 487: 1887, 1887, 1887, 1887, 1887, 494: 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 508: 1887, 510: 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 531: 1887, 1887, 720: 3230, 3228}, {1264, 1264, 7: 1264, 57: 1264, 124: 1264, 458: 1264, 460: 1264, 466: 1264, 468: 1264, 476: 1264, 1264, 479: 1264, 1264, 1264, 483: 1264, 488: 1264, 490: 1264, 501: 1264, 1264, 510: 1264, 513: 1264, 1264}, // 1605 @@ -7944,7 +7944,7 @@ var ( {238, 238, 57: 238, 458: 238, 460: 238, 466: 238, 468: 238, 476: 238, 238, 479: 238, 238, 238, 483: 238, 488: 238, 490: 238, 3234, 494: 3232, 3233, 3231, 3229, 499: 238, 501: 238, 238, 720: 3230, 3228}, {4, 4}, {143: 4066}, - {237, 237, 480: 237, 488: 237, 2624, 237, 778: 2625, 4067}, + {237, 237, 480: 237, 488: 237, 2624, 237, 777: 2625, 4067}, {1258, 1258, 480: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 4068}, // 1620 {834, 834, 480: 2630, 488: 2631, 754: 2632, 815: 4069}, @@ -7953,8 +7953,8 @@ var ( {2: 1802, 1802, 1802, 1802, 1802, 8: 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 58: 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 1802, 459: 1802, 481: 1802, 487: 1802, 553: 1802, 565: 1802}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 4073}, // 1625 - {2329, 2329, 2329, 2329, 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 80: 4102, 83: 4123, 4124, 92: 4125, 132: 4105, 190: 4090, 4109, 194: 4110, 206: 4104, 213: 4120, 224: 4099, 234: 4106, 238: 4101, 253: 4111, 262: 4107, 269: 4121, 4122, 276: 4091, 460: 4119, 464: 4130, 466: 4167, 2106, 475: 2329, 483: 4126, 485: 4118, 2106, 490: 4108, 498: 4093, 571: 4098, 4094, 634: 2106, 4136, 640: 4075, 653: 4113, 656: 4100, 658: 4127, 666: 4112, 673: 4114, 676: 4095, 691: 4103, 758: 4087, 762: 4088, 764: 4141, 780: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 850: 4117, 861: 4115, 903: 4092, 909: 4096, 969: 4129, 1113: 4097, 1141: 4116, 1146: 4128, 4074}, - {2104, 2104, 4912, 4913, 475: 4914, 1076: 4911, 1145: 4910}, + {2329, 2329, 2329, 2329, 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 80: 4102, 83: 4123, 4124, 92: 4125, 132: 4105, 190: 4090, 4109, 194: 4110, 206: 4104, 213: 4120, 224: 4099, 234: 4106, 238: 4101, 253: 4111, 262: 4107, 269: 4121, 4122, 276: 4091, 460: 4119, 464: 4130, 466: 4167, 2106, 475: 2329, 483: 4126, 485: 4118, 2106, 490: 4108, 498: 4093, 571: 4098, 4094, 634: 2106, 4136, 640: 4075, 653: 4113, 656: 4100, 658: 4127, 666: 4112, 673: 4114, 676: 4095, 691: 4103, 758: 4087, 762: 4088, 764: 4141, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 850: 4117, 862: 4115, 904: 4092, 910: 4096, 970: 4129, 1114: 4097, 1142: 4116, 1147: 4128, 4074}, + {2104, 2104, 4912, 4913, 475: 4914, 1077: 4911, 1146: 4910}, {475: 4884}, {461: 1987, 482: 4171, 723: 4882}, {461: 1987, 482: 4171, 723: 4880}, @@ -7975,17 +7975,17 @@ var ( {154: 4854}, {461: 1987, 464: 1987, 482: 4171, 723: 4851}, {461: 1987, 464: 1987, 482: 4171, 723: 4848}, - {2412, 2412, 2412, 2412, 4131, 4133, 389, 2412, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 464: 4130, 466: 4167, 2106, 475: 2412, 485: 4844, 2106, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 780: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4845}, + {2412, 2412, 2412, 2412, 4131, 4133, 389, 2412, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 464: 4130, 466: 4167, 2106, 475: 2412, 485: 4844, 2106, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4845}, // 1645 {382: 4834}, {637: 4826}, - {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 459: 2334, 475: 4685, 552: 2334, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 662: 2323, 2323, 819: 4687, 830: 4313, 853: 4683, 884: 4684, 898: 4686}, + {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 459: 2334, 475: 4685, 552: 2334, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 662: 2323, 2323, 819: 4687, 830: 4313, 853: 4683, 885: 4684, 899: 4686}, {2402, 2402, 2402, 2402, 7: 2402, 475: 2402}, {2401, 2401, 2401, 2401, 7: 2401, 475: 2401}, // 1650 {475: 4681}, {475: 4678}, - {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 475: 4657, 552: 2334, 563: 4299, 571: 4314, 573: 4656, 630: 4315, 635: 4300, 662: 4660, 813: 4659, 830: 4313, 853: 4655, 898: 4658, 980: 4661}, + {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 475: 4657, 552: 2334, 563: 4299, 571: 4314, 573: 4656, 630: 4315, 635: 4300, 662: 4660, 813: 4659, 830: 4313, 853: 4655, 899: 4658, 981: 4661}, {475: 4644}, {475: 4642}, // 1655 @@ -7996,12 +7996,12 @@ var ( {475: 4618}, // 1660 {646: 4611}, - {926: 4610}, - {926: 4609}, + {927: 4610}, + {927: 4609}, {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 552: 2334, 830: 4313, 853: 4605}, {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 552: 2334, 830: 4313, 853: 4339}, // 1665 - {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 571: 4314, 630: 4315, 635: 4312, 830: 4313, 853: 4310, 980: 4311}, + {2: 2334, 2334, 2334, 2334, 2334, 8: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 58: 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 2334, 571: 4314, 630: 4315, 635: 4312, 830: 4313, 853: 4310, 981: 4311}, {2: 1987, 1987, 1987, 1987, 1987, 8: 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 58: 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 465: 4297, 482: 4171, 563: 4299, 635: 4300, 637: 4295, 723: 4296, 813: 4298, 830: 4294}, {2370, 2370, 2370, 2370, 7: 2370, 475: 2370}, {2369, 2369, 2369, 2369, 7: 2369, 475: 2369}, @@ -8089,12 +8089,12 @@ var ( {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, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 458: 366, 366, 366, 464: 366, 366, 366, 366, 474: 366, 366, 484: 366, 366, 366, 492: 366, 553: 366, 632: 366, 634: 366, 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, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 458: 365, 365, 365, 464: 365, 365, 365, 365, 474: 365, 365, 484: 365, 365, 365, 492: 365, 553: 365, 632: 365, 634: 365, 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, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 458: 364, 364, 364, 464: 364, 364, 364, 364, 474: 364, 364, 484: 364, 364, 364, 492: 364, 553: 364, 632: 364, 634: 364, 364}, - {461: 4189, 1019: 4188}, + {461: 4189, 1020: 4188}, // 1740 {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, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 458: 390, 390, 390, 464: 390, 390, 390, 390, 474: 390, 390, 484: 390, 390, 390, 492: 390, 553: 390, 632: 390, 634: 390, 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, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 458: 9, 9, 9, 464: 9, 9, 9, 9, 474: 9, 9, 484: 9, 9, 9, 492: 9, 553: 9, 632: 9, 634: 9, 9}, {459: 4191}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 547, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 4192, 1122: 4193}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 547, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 4192, 1123: 4193}, {546, 546, 7: 3808, 57: 546, 460: 546}, // 1745 {57: 4194}, @@ -8103,7 +8103,7 @@ var ( {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, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 458: 393, 393, 393, 464: 393, 393, 393, 393, 474: 393, 393, 484: 393, 393, 393, 492: 393, 553: 393, 632: 393, 634: 393, 393}, {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, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 458: 392, 392, 392, 464: 392, 392, 392, 392, 474: 392, 392, 484: 392, 392, 392, 492: 392, 553: 392, 632: 392, 634: 392, 392}, // 1750 - {464: 4200, 493: 2638, 722: 2637, 730: 4201, 1114: 4199}, + {464: 4200, 493: 2638, 722: 2637, 730: 4201, 1115: 4199}, {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, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 458: 396, 396, 396, 464: 396, 396, 396, 396, 474: 396, 396, 484: 396, 396, 396, 492: 396, 553: 396, 632: 396, 634: 396, 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, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 458: 387, 387, 387, 464: 387, 387, 387, 387, 474: 387, 387, 484: 387, 387, 387, 492: 387, 553: 387, 632: 387, 634: 387, 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, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 458: 386, 386, 386, 464: 386, 386, 386, 386, 474: 386, 386, 484: 386, 386, 386, 492: 386, 553: 386, 632: 386, 634: 386, 386}, @@ -8131,7 +8131,7 @@ var ( {464: 4221, 493: 2638, 722: 2637, 730: 4220}, {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, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 458: 407, 407, 407, 464: 407, 407, 407, 407, 474: 407, 407, 484: 407, 407, 407, 492: 407, 553: 407, 632: 407, 634: 407, 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, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 458: 406, 406, 406, 464: 406, 406, 406, 406, 474: 406, 406, 484: 406, 406, 406, 492: 406, 553: 406, 632: 406, 634: 406, 406}, - {464: 4200, 493: 2638, 722: 2637, 730: 4201, 1114: 4223}, + {464: 4200, 493: 2638, 722: 2637, 730: 4201, 1115: 4223}, // 1775 {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, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 458: 408, 408, 408, 464: 408, 408, 408, 408, 474: 408, 408, 484: 408, 408, 408, 492: 408, 553: 408, 632: 408, 634: 408, 408}, {493: 2638, 722: 2637, 730: 4225}, @@ -8167,7 +8167,7 @@ var ( {2: 1987, 1987, 1987, 1987, 1987, 8: 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 58: 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 461: 1987, 482: 4171, 529: 1987, 723: 4250}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3375, 648: 3377, 2664, 2665, 2663, 725: 3374, 852: 4251}, {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, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 458: 421, 421, 421, 464: 421, 421, 421, 421, 474: 421, 421, 484: 421, 421, 421, 492: 421, 553: 421, 632: 421, 634: 421, 421}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 4253}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 4253}, // 1805 {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, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 458: 422, 422, 422, 464: 422, 422, 422, 422, 474: 422, 422, 484: 422, 422, 422, 492: 422, 553: 422, 632: 422, 634: 422, 422}, {493: 2638, 722: 2637, 730: 4255}, @@ -8200,7 +8200,7 @@ var ( {461: 4278}, // 1830 {2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 13: 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 2070, 458: 2070, 2070, 2070, 464: 2070, 2070, 2070, 2070, 474: 2070, 2070, 484: 2070, 2070, 2070, 492: 2070, 553: 2070, 632: 2070, 634: 2070, 2070}, - {4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 80: 4102, 83: 4123, 4124, 92: 4125, 132: 4105, 190: 4090, 4109, 194: 4110, 206: 4104, 213: 4120, 224: 4099, 234: 4106, 238: 4101, 253: 4111, 262: 4107, 269: 4121, 4122, 276: 4091, 460: 4119, 464: 4130, 466: 4167, 2106, 483: 4126, 485: 4118, 2106, 490: 4108, 498: 4093, 571: 4098, 4094, 634: 2106, 4136, 653: 4113, 656: 4100, 658: 4127, 666: 4112, 673: 4114, 676: 4095, 691: 4103, 758: 4087, 762: 4088, 764: 4141, 780: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 850: 4117, 861: 4115, 903: 4092, 909: 4096, 969: 4280, 1113: 4097, 1141: 4116}, + {4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 80: 4102, 83: 4123, 4124, 92: 4125, 132: 4105, 190: 4090, 4109, 194: 4110, 206: 4104, 213: 4120, 224: 4099, 234: 4106, 238: 4101, 253: 4111, 262: 4107, 269: 4121, 4122, 276: 4091, 460: 4119, 464: 4130, 466: 4167, 2106, 483: 4126, 485: 4118, 2106, 490: 4108, 498: 4093, 571: 4098, 4094, 634: 2106, 4136, 653: 4113, 656: 4100, 658: 4127, 666: 4112, 673: 4114, 676: 4095, 691: 4103, 758: 4087, 762: 4088, 764: 4141, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 850: 4117, 862: 4115, 904: 4092, 910: 4096, 970: 4280, 1114: 4097, 1142: 4116}, {2326, 2326, 2326, 2326, 7: 2326, 475: 2326}, {2340, 2340, 2340, 2340, 7: 2340, 475: 2340}, {2339, 2339, 2339, 2339, 7: 2339, 475: 2339}, @@ -8249,7 +8249,7 @@ var ( // 1870 {1781, 1781, 1781, 1781, 1781, 7: 1781, 29: 1781, 57: 1781, 92: 1781, 1781, 1781, 1781, 1781, 1781, 460: 1781, 468: 1781, 475: 1781, 483: 1781}, {1780, 1780, 1780, 1780, 1780, 7: 1780, 29: 1780, 57: 1780, 92: 1780, 1780, 1780, 1780, 1780, 1780, 460: 1780, 468: 1780, 475: 1780, 483: 1780}, - {146: 4322, 462: 3970, 3969, 796: 4323, 917: 4321}, + {146: 4322, 462: 3970, 3969, 796: 4323, 918: 4321}, {2362, 2362, 2362, 2362, 7: 2362, 475: 2362}, {2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 57: 2230, 458: 2230, 462: 2230, 2230, 2230, 2230, 467: 2230, 475: 2230, 478: 2230, 563: 2230, 571: 2230, 573: 2230, 629: 2230, 2230, 2230, 633: 2230}, // 1875 @@ -8260,7 +8260,7 @@ var ( {464: 4328}, // 1880 {2376, 2376, 2376, 2376, 7: 2376, 475: 2376}, - {459: 4331, 461: 3127, 471: 4334, 4333, 478: 3118, 493: 3122, 557: 3117, 3119, 3121, 3120, 562: 3125, 566: 3126, 579: 3124, 699: 4332, 3123, 1109: 4330}, + {459: 4331, 461: 3127, 471: 4334, 4333, 478: 3118, 493: 3122, 557: 3117, 3119, 3121, 3120, 562: 3125, 566: 3126, 579: 3124, 699: 4332, 3123, 1110: 4330}, {2378, 2378, 2378, 2378, 7: 2378, 475: 2378}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 4337}, {2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 57: 2152, 458: 2152, 462: 2152, 2152, 2152, 2152, 467: 2152, 475: 2152, 478: 2152, 563: 2152, 571: 2152, 573: 2152, 629: 2152, 2152, 2152, 633: 2152}, @@ -8278,10 +8278,10 @@ var ( {2: 1806, 1806, 1806, 1806, 1806, 8: 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 58: 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 1806, 461: 1806, 556: 1806}, // 1895 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4345, 831: 4344}, - {2332, 2332, 2332, 2332, 7: 2332, 4602, 4603, 475: 2332, 912: 4601}, - {10: 4347, 105: 4395, 109: 4396, 170: 4406, 4405, 4371, 174: 4386, 187: 4408, 212: 4407, 218: 4368, 296: 4375, 4367, 317: 4384, 343: 4391, 4390, 347: 4394, 380: 4402, 486: 4389, 498: 4385, 529: 4380, 634: 4388, 664: 4393, 4392, 667: 4369, 4374, 4372, 4365, 4359, 4373, 675: 4381, 677: 4366, 4398, 4360, 4361, 4362, 4363, 4364, 4387, 4400, 4404, 4399, 4358, 4403, 4370, 692: 4357, 4397, 4356, 4401, 881: 4376, 1133: 4378, 1155: 4355, 4382, 4352, 1175: 4350, 1189: 4353, 1191: 4354, 1211: 4351, 1228: 4377, 4348, 4379, 1285: 4349, 1297: 4383, 1300: 4346, 1325: 4409}, - {2193, 2193, 2193, 2193, 4489, 4495, 4483, 2193, 2193, 2193, 4487, 4496, 4494, 57: 2193, 458: 4488, 462: 3970, 3969, 4486, 2200, 467: 4493, 475: 2193, 478: 4482, 563: 2234, 571: 2323, 573: 4480, 629: 4485, 4478, 4500, 633: 4497, 796: 4481, 819: 4490, 892: 4492, 911: 4498, 920: 4491, 936: 4484, 983: 4499, 4600}, - {2193, 2193, 2193, 2193, 4489, 4495, 4483, 2193, 2193, 2193, 4487, 4496, 4494, 57: 2193, 458: 4488, 462: 3970, 3969, 4486, 2200, 467: 4493, 475: 2193, 478: 4482, 563: 2234, 571: 2323, 573: 4480, 629: 4485, 4478, 4500, 633: 4497, 796: 4481, 819: 4490, 892: 4492, 911: 4498, 920: 4491, 936: 4484, 983: 4499, 4479}, + {2332, 2332, 2332, 2332, 7: 2332, 4602, 4603, 475: 2332, 913: 4601}, + {10: 4347, 105: 4395, 109: 4396, 170: 4406, 4405, 4371, 174: 4386, 187: 4408, 212: 4407, 218: 4368, 296: 4375, 4367, 317: 4384, 343: 4391, 4390, 347: 4394, 380: 4402, 486: 4389, 498: 4385, 529: 4380, 634: 4388, 664: 4393, 4392, 667: 4369, 4374, 4372, 4365, 4359, 4373, 675: 4381, 677: 4366, 4398, 4360, 4361, 4362, 4363, 4364, 4387, 4400, 4404, 4399, 4358, 4403, 4370, 692: 4357, 4397, 4356, 4401, 882: 4376, 1134: 4378, 1156: 4355, 4382, 4352, 1176: 4350, 1190: 4353, 1192: 4354, 1211: 4351, 1228: 4377, 4348, 4379, 1285: 4349, 1297: 4383, 1300: 4346, 1325: 4409}, + {2193, 2193, 2193, 2193, 4489, 4495, 4483, 2193, 2193, 2193, 4487, 4496, 4494, 57: 2193, 458: 4488, 462: 3970, 3969, 4486, 2200, 467: 4493, 475: 2193, 478: 4482, 563: 2234, 571: 2323, 573: 4480, 629: 4485, 4478, 4500, 633: 4497, 796: 4481, 819: 4490, 893: 4492, 912: 4498, 921: 4491, 937: 4484, 984: 4499, 4600}, + {2193, 2193, 2193, 2193, 4489, 4495, 4483, 2193, 2193, 2193, 4487, 4496, 4494, 57: 2193, 458: 4488, 462: 3970, 3969, 4486, 2200, 467: 4493, 475: 2193, 478: 4482, 563: 2234, 571: 2323, 573: 4480, 629: 4485, 4478, 4500, 633: 4497, 796: 4481, 819: 4490, 893: 4492, 912: 4498, 921: 4491, 937: 4484, 984: 4499, 4479}, // 1900 {363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 57: 363, 458: 363, 462: 363, 363, 363, 363, 467: 363, 475: 363, 478: 363, 563: 363, 571: 363, 573: 363, 629: 363, 363, 363, 633: 363}, {362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 57: 362, 458: 362, 462: 362, 362, 362, 362, 467: 362, 475: 362, 478: 362, 563: 362, 571: 362, 573: 362, 629: 362, 362, 362, 633: 362}, @@ -8289,8 +8289,8 @@ var ( {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 60: 278, 458: 278, 3710, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 726: 278, 728: 278, 747: 3711, 770: 4476}, {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 57: 273, 60: 273, 458: 273, 462: 273, 273, 273, 273, 467: 273, 475: 273, 478: 273, 563: 273, 571: 273, 573: 273, 629: 273, 273, 273, 633: 273, 726: 273, 728: 273, 855: 4475}, // 1905 - {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 57: 271, 60: 271, 458: 271, 3697, 462: 271, 271, 271, 271, 467: 271, 475: 271, 478: 271, 563: 271, 571: 271, 573: 271, 629: 271, 271, 271, 633: 271, 726: 271, 728: 271, 747: 3698, 885: 4473, 891: 3699}, - {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 57: 271, 60: 271, 458: 271, 3697, 462: 271, 271, 271, 271, 467: 271, 475: 271, 478: 271, 563: 271, 571: 271, 573: 271, 629: 271, 271, 271, 633: 271, 726: 271, 728: 271, 747: 3698, 885: 4471, 891: 3699}, + {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 57: 271, 60: 271, 458: 271, 3697, 462: 271, 271, 271, 271, 467: 271, 475: 271, 478: 271, 563: 271, 571: 271, 573: 271, 629: 271, 271, 271, 633: 271, 726: 271, 728: 271, 747: 3698, 886: 4473, 892: 3699}, + {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 57: 271, 60: 271, 458: 271, 3697, 462: 271, 271, 271, 271, 467: 271, 475: 271, 478: 271, 563: 271, 571: 271, 573: 271, 629: 271, 271, 271, 633: 271, 726: 271, 728: 271, 747: 3698, 886: 4471, 892: 3699}, {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3710, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3711, 770: 4470}, {355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 57: 355, 60: 355, 458: 355, 355, 462: 355, 355, 355, 355, 467: 355, 475: 355, 478: 355, 563: 355, 571: 355, 573: 355, 629: 355, 355, 355, 633: 355, 726: 355, 728: 355}, {354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 57: 354, 60: 354, 458: 354, 354, 462: 354, 354, 354, 354, 467: 354, 475: 354, 478: 354, 563: 354, 571: 354, 573: 354, 629: 354, 354, 354, 633: 354, 726: 354, 728: 354}, @@ -8316,8 +8316,8 @@ var ( {335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 57: 335, 60: 335, 458: 335, 335, 462: 335, 335, 335, 335, 467: 335, 475: 335, 478: 335, 563: 335, 571: 335, 573: 335, 629: 335, 335, 335, 633: 335, 726: 335, 728: 335}, {334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 57: 334, 60: 334, 458: 334, 334, 462: 334, 334, 334, 334, 467: 334, 475: 334, 478: 334, 563: 334, 571: 334, 573: 334, 629: 334, 334, 334, 633: 334, 726: 334, 728: 334, 1252: 4469}, {332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 57: 332, 458: 332, 332, 462: 332, 332, 332, 332, 467: 332, 475: 332, 478: 332, 563: 332, 571: 332, 573: 332, 629: 332, 332, 332, 633: 332}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 3710, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 747: 4466, 759: 3718, 785: 4467}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 3710, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 747: 4463, 759: 3718, 785: 4464}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 3710, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 747: 4466, 759: 3718, 784: 4467}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 3710, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 747: 4463, 759: 3718, 784: 4464}, // 1930 {459: 3710, 747: 4461}, {459: 3710, 747: 4459}, @@ -8325,11 +8325,11 @@ var ( {459: 3710, 747: 4457}, {323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 57: 323, 458: 323, 462: 323, 323, 323, 323, 467: 323, 475: 323, 478: 323, 563: 323, 571: 323, 573: 323, 629: 323, 323, 323, 633: 323}, // 1935 - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4437, 862: 4456}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4437, 863: 4456}, {459: 4452}, {459: 4442}, {319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 57: 319, 458: 319, 462: 319, 319, 319, 319, 467: 319, 475: 319, 478: 319, 563: 319, 571: 319, 573: 319, 629: 319, 319, 319, 633: 319}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 4435, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 4434, 664: 4393, 4392, 675: 4436, 759: 3718, 785: 4437, 862: 4433, 1133: 4432}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 4435, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 4434, 664: 4393, 4392, 675: 4436, 759: 3718, 784: 4437, 863: 4433, 1134: 4432}, // 1940 {316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 57: 316, 458: 316, 316, 462: 316, 316, 316, 316, 467: 316, 475: 316, 478: 316, 486: 316, 529: 316, 563: 316, 571: 316, 573: 316, 629: 316, 316, 316, 633: 316, 316, 829: 4431}, {315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 57: 315, 458: 315, 315, 462: 315, 315, 315, 315, 467: 315, 475: 315, 478: 315, 486: 315, 529: 315, 563: 315, 571: 315, 573: 315, 629: 315, 315, 315, 633: 315, 315, 829: 4430}, @@ -8383,7 +8383,7 @@ var ( {459: 300}, {310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 57: 310, 101: 310, 310, 104: 310, 458: 310, 310, 462: 310, 310, 310, 310, 467: 310, 475: 310, 478: 310, 486: 310, 529: 310, 563: 310, 571: 310, 573: 310, 629: 310, 310, 310, 633: 310, 310}, {311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 57: 311, 101: 311, 311, 104: 311, 458: 311, 311, 462: 311, 311, 311, 311, 467: 311, 475: 311, 478: 311, 486: 311, 529: 311, 563: 311, 571: 311, 573: 311, 629: 311, 311, 311, 633: 311, 311}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4437, 862: 4441}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4437, 863: 4441}, // 1985 {317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 57: 317, 458: 317, 462: 317, 317, 317, 317, 467: 317, 475: 317, 478: 317, 563: 317, 571: 317, 573: 317, 629: 317, 317, 317, 633: 317}, {498: 3723, 829: 4431}, @@ -8395,7 +8395,7 @@ var ( {286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 57: 286, 458: 286, 462: 286, 286, 286, 286, 467: 286, 475: 286, 478: 286, 563: 286, 571: 286, 573: 286, 629: 286, 286, 286, 633: 286}, {285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 57: 285, 458: 285, 462: 285, 285, 285, 285, 467: 285, 475: 285, 478: 285, 563: 285, 571: 285, 573: 285, 629: 285, 285, 285, 633: 285}, {318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 57: 318, 458: 318, 462: 318, 318, 318, 318, 467: 318, 475: 318, 478: 318, 563: 318, 571: 318, 573: 318, 629: 318, 318, 318, 633: 318}, - {461: 4444, 562: 4445, 566: 4446, 950: 4447, 1126: 4443}, + {461: 4444, 562: 4445, 566: 4446, 951: 4447, 1127: 4443}, // 1995 {7: 4449, 57: 4448}, {7: 253, 57: 253}, @@ -8403,28 +8403,28 @@ var ( {7: 251, 57: 251}, {7: 250, 57: 250}, // 2000 - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4437, 862: 4451}, - {461: 4444, 562: 4445, 566: 4446, 950: 4450}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4437, 863: 4451}, + {461: 4444, 562: 4445, 566: 4446, 951: 4450}, {7: 249, 57: 249}, {320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 57: 320, 458: 320, 462: 320, 320, 320, 320, 467: 320, 475: 320, 478: 320, 563: 320, 571: 320, 573: 320, 629: 320, 320, 320, 633: 320}, - {461: 4444, 562: 4445, 566: 4446, 950: 4447, 1126: 4453}, + {461: 4444, 562: 4445, 566: 4446, 951: 4447, 1127: 4453}, // 2005 {7: 4449, 57: 4454}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4437, 862: 4455}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 101: 4438, 4440, 104: 4439, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4437, 863: 4455}, {321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 57: 321, 458: 321, 462: 321, 321, 321, 321, 467: 321, 475: 321, 478: 321, 563: 321, 571: 321, 573: 321, 629: 321, 321, 321, 633: 321}, {322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 57: 322, 458: 322, 462: 322, 322, 322, 322, 467: 322, 475: 322, 478: 322, 563: 322, 571: 322, 573: 322, 629: 322, 322, 322, 633: 322}, {324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 57: 324, 458: 324, 462: 324, 324, 324, 324, 467: 324, 475: 324, 478: 324, 563: 324, 571: 324, 573: 324, 629: 324, 324, 324, 633: 324}, // 2010 {325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 57: 325, 458: 325, 462: 325, 325, 325, 325, 467: 325, 475: 325, 478: 325, 563: 325, 571: 325, 573: 325, 629: 325, 325, 325, 633: 325}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4460}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4460}, {326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 57: 326, 458: 326, 462: 326, 326, 326, 326, 467: 326, 475: 326, 478: 326, 563: 326, 571: 326, 573: 326, 629: 326, 326, 326, 633: 326}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4462}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4462}, {327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 57: 327, 458: 327, 462: 327, 327, 327, 327, 467: 327, 475: 327, 478: 327, 563: 327, 571: 327, 573: 327, 629: 327, 327, 327, 633: 327}, // 2015 - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4465}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4465}, {328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 57: 328, 458: 328, 462: 328, 328, 328, 328, 467: 328, 475: 328, 478: 328, 563: 328, 571: 328, 573: 328, 629: 328, 328, 328, 633: 328}, {329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 57: 329, 458: 329, 462: 329, 329, 329, 329, 467: 329, 475: 329, 478: 329, 563: 329, 571: 329, 573: 329, 629: 329, 329, 329, 633: 329}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 785: 4468}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3720, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3721, 529: 3717, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3719, 759: 3718, 784: 4468}, {330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 57: 330, 458: 330, 462: 330, 330, 330, 330, 467: 330, 475: 330, 478: 330, 563: 330, 571: 330, 573: 330, 629: 330, 330, 330, 633: 330}, // 2020 {331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 57: 331, 458: 331, 462: 331, 331, 331, 331, 467: 331, 475: 331, 478: 331, 563: 331, 571: 331, 573: 331, 629: 331, 331, 331, 633: 331}, @@ -8439,7 +8439,7 @@ var ( {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 57: 273, 60: 273, 458: 273, 462: 273, 273, 273, 273, 467: 273, 475: 273, 478: 273, 563: 273, 571: 273, 573: 273, 629: 273, 273, 273, 633: 273, 726: 273, 728: 273, 855: 4477}, {360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 57: 360, 60: 4413, 458: 360, 462: 360, 360, 360, 360, 467: 360, 475: 360, 478: 360, 563: 360, 571: 360, 573: 360, 629: 360, 360, 360, 633: 360, 726: 4412, 728: 4414, 854: 4415}, // 2030 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 563: 2322, 571: 2322, 573: 2322, 629: 2322, 635: 2322, 648: 4599, 2664, 2665, 2663, 662: 2322, 2322, 1119: 4598}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 563: 2322, 571: 2322, 573: 2322, 629: 2322, 635: 2322, 648: 4599, 2664, 2665, 2663, 662: 2322, 2322, 1120: 4598}, {2256, 2256, 2256, 2256, 7: 2256, 2256, 2256, 57: 2256, 475: 2256}, {563: 2233}, {478: 4597}, @@ -8448,7 +8448,7 @@ var ( {2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 57: 2222, 458: 2222, 462: 2222, 2222, 2222, 2222, 467: 2222, 475: 2222, 478: 2222, 563: 2222, 571: 2222, 573: 2222, 629: 2222, 2222, 2222, 633: 2222}, {563: 4593}, {2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 57: 2219, 458: 2219, 462: 2219, 2219, 2219, 2219, 467: 2219, 475: 2219, 478: 2219, 563: 4592, 571: 2219, 573: 2219, 629: 2219, 2219, 2219, 633: 2219}, - {254: 4590, 345: 4591, 461: 3127, 471: 4334, 4333, 478: 3118, 493: 3122, 557: 3117, 3119, 3121, 3120, 562: 3125, 566: 3126, 575: 4579, 4576, 4577, 4578, 3124, 699: 4332, 3123, 4589, 1056: 4574, 4575, 4587, 1109: 4588, 1177: 4586}, + {254: 4590, 345: 4591, 461: 3127, 471: 4334, 4333, 478: 3118, 493: 3122, 557: 3117, 3119, 3121, 3120, 562: 3125, 566: 3126, 575: 4579, 4576, 4577, 4578, 3124, 699: 4332, 3123, 4589, 1057: 4574, 4575, 4587, 1110: 4588, 1178: 4586}, {464: 4584}, // 2040 {641: 4572}, @@ -8458,13 +8458,13 @@ var ( {2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 57: 2211, 458: 2211, 462: 2211, 2211, 2211, 2211, 467: 2211, 475: 2211, 478: 2211, 563: 2211, 571: 2211, 573: 2211, 629: 2211, 2211, 2211, 633: 2211}, // 2045 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3375, 648: 3377, 2664, 2665, 2663, 725: 3374, 852: 4554}, - {172: 4552, 193: 4553, 464: 4551, 1162: 4550}, + {172: 4552, 193: 4553, 464: 4551, 1163: 4550}, {176: 4549, 235: 4548, 464: 4547, 1281: 4546}, {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3710, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3711, 770: 4545}, {293: 4544}, // 2050 {2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 57: 2195, 458: 2195, 462: 2195, 2195, 2195, 2195, 467: 2195, 475: 2195, 478: 2195, 563: 2195, 571: 2195, 573: 2195, 629: 2195, 2195, 2195, 633: 2195}, - {2192, 2192, 2192, 2192, 4489, 4495, 4483, 2192, 2192, 2192, 4487, 4496, 4494, 57: 2192, 458: 4488, 462: 3970, 3969, 4486, 2200, 467: 4493, 475: 2192, 478: 4482, 563: 2234, 571: 2323, 573: 4480, 629: 4485, 4478, 4500, 633: 4497, 796: 4481, 819: 4490, 892: 4492, 911: 4543, 920: 4491, 936: 4484}, + {2192, 2192, 2192, 2192, 4489, 4495, 4483, 2192, 2192, 2192, 4487, 4496, 4494, 57: 2192, 458: 4488, 462: 3970, 3969, 4486, 2200, 467: 4493, 475: 2192, 478: 4482, 563: 2234, 571: 2323, 573: 4480, 629: 4485, 4478, 4500, 633: 4497, 796: 4481, 819: 4490, 893: 4492, 912: 4543, 921: 4491, 937: 4484}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 4501}, {2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 57: 2140, 458: 2140, 4503, 462: 2140, 2140, 2140, 2140, 467: 2140, 475: 2140, 478: 2140, 563: 2140, 571: 2140, 573: 2140, 629: 2140, 2140, 2140, 633: 2140, 636: 2140, 1207: 4502}, {2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 57: 2182, 458: 2182, 462: 2182, 2182, 2182, 2182, 467: 2182, 475: 2182, 478: 2182, 563: 2182, 571: 2182, 573: 2182, 629: 2182, 2182, 2182, 633: 2182, 636: 4518, 1224: 4519, 4520}, @@ -8476,12 +8476,12 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 4508}, // 2060 {57: 4509, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {7: 1261, 57: 1261, 515: 4512, 4511, 929: 4510}, + {7: 1261, 57: 1261, 515: 4512, 4511, 930: 4510}, {7: 2135, 57: 2135}, {1260, 1260, 1260, 1260, 7: 1260, 57: 1260, 475: 1260}, {1259, 1259, 1259, 1259, 7: 1259, 57: 1259, 475: 1259}, // 2065 - {7: 1261, 57: 1261, 515: 4512, 4511, 929: 4514}, + {7: 1261, 57: 1261, 515: 4512, 4511, 930: 4514}, {7: 2136, 57: 2136}, {2139, 2139, 2139, 2139, 2139, 2139, 2139, 2139, 2139, 2139, 2139, 2139, 2139, 57: 2139, 458: 2139, 462: 2139, 2139, 2139, 2139, 467: 2139, 475: 2139, 478: 2139, 563: 2139, 571: 2139, 573: 2139, 629: 2139, 2139, 2139, 633: 2139, 636: 2139}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4507, 648: 4024, 2664, 2665, 2663, 729: 4506, 812: 4517}, @@ -8489,15 +8489,15 @@ var ( // 2070 {197: 4540, 352: 4541, 369: 4542}, {2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 57: 2181, 458: 2181, 462: 2181, 2181, 2181, 2181, 467: 2181, 475: 2181, 478: 2181, 563: 2181, 571: 2181, 573: 2181, 629: 2181, 2181, 2181, 633: 2181}, - {2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 57: 2177, 458: 4522, 462: 2177, 2177, 2177, 2177, 467: 2177, 475: 2177, 478: 2177, 563: 2177, 571: 2177, 573: 2177, 629: 2177, 2177, 2177, 633: 2177, 1064: 4523, 4524, 1231: 4521}, + {2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 57: 2177, 458: 4522, 462: 2177, 2177, 2177, 2177, 467: 2177, 475: 2177, 478: 2177, 563: 2177, 571: 2177, 573: 2177, 629: 2177, 2177, 2177, 633: 2177, 1065: 4523, 4524, 1231: 4521}, {2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 57: 2180, 458: 2180, 462: 2180, 2180, 2180, 2180, 467: 2180, 475: 2180, 478: 2180, 563: 2180, 571: 2180, 573: 2180, 629: 2180, 2180, 2180, 633: 2180}, {641: 4538, 731: 4527}, // 2075 - {2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 57: 2176, 458: 4536, 462: 2176, 2176, 2176, 2176, 467: 2176, 475: 2176, 478: 2176, 563: 2176, 571: 2176, 573: 2176, 629: 2176, 2176, 2176, 633: 2176, 1065: 4537}, - {2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 57: 2175, 458: 4525, 462: 2175, 2175, 2175, 2175, 467: 2175, 475: 2175, 478: 2175, 563: 2175, 571: 2175, 573: 2175, 629: 2175, 2175, 2175, 633: 2175, 1064: 4526}, + {2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 57: 2176, 458: 4536, 462: 2176, 2176, 2176, 2176, 467: 2176, 475: 2176, 478: 2176, 563: 2176, 571: 2176, 573: 2176, 629: 2176, 2176, 2176, 633: 2176, 1066: 4537}, + {2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 57: 2175, 458: 4525, 462: 2175, 2175, 2175, 2175, 467: 2175, 475: 2175, 478: 2175, 563: 2175, 571: 2175, 573: 2175, 629: 2175, 2175, 2175, 633: 2175, 1065: 4526}, {731: 4527}, {2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 57: 2173, 458: 2173, 462: 2173, 2173, 2173, 2173, 467: 2173, 475: 2173, 478: 2173, 563: 2173, 571: 2173, 573: 2173, 629: 2173, 2173, 2173, 633: 2173}, - {81: 4532, 498: 4531, 657: 4530, 659: 4529, 1087: 4528}, + {81: 4532, 498: 4531, 657: 4530, 659: 4529, 1088: 4528}, // 2080 {2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 57: 2179, 458: 2179, 462: 2179, 2179, 2179, 2179, 467: 2179, 475: 2179, 478: 2179, 563: 2179, 571: 2179, 573: 2179, 629: 2179, 2179, 2179, 633: 2179}, {2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 57: 2172, 458: 2172, 462: 2172, 2172, 2172, 2172, 467: 2172, 475: 2172, 478: 2172, 563: 2172, 571: 2172, 573: 2172, 629: 2172, 2172, 2172, 633: 2172}, @@ -8511,7 +8511,7 @@ var ( {641: 4538}, {2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 57: 2174, 458: 2174, 462: 2174, 2174, 2174, 2174, 467: 2174, 475: 2174, 478: 2174, 563: 2174, 571: 2174, 573: 2174, 629: 2174, 2174, 2174, 633: 2174}, // 2090 - {81: 4532, 498: 4531, 657: 4530, 659: 4529, 1087: 4539}, + {81: 4532, 498: 4531, 657: 4530, 659: 4529, 1088: 4539}, {2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 57: 2178, 458: 2178, 462: 2178, 2178, 2178, 2178, 467: 2178, 475: 2178, 478: 2178, 563: 2178, 571: 2178, 573: 2178, 629: 2178, 2178, 2178, 633: 2178}, {2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 57: 2185, 458: 2185, 462: 2185, 2185, 2185, 2185, 467: 2185, 475: 2185, 478: 2185, 563: 2185, 571: 2185, 573: 2185, 629: 2185, 2185, 2185, 633: 2185}, {2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 57: 2184, 458: 2184, 462: 2184, 2184, 2184, 2184, 467: 2184, 475: 2184, 478: 2184, 563: 2184, 571: 2184, 573: 2184, 629: 2184, 2184, 2184, 633: 2184}, @@ -8543,7 +8543,7 @@ var ( // 2115 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 4564}, {57: 4565, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 57: 2228, 146: 4322, 458: 2228, 462: 3970, 3969, 2228, 2228, 467: 2228, 475: 2228, 478: 2228, 563: 2228, 571: 2228, 573: 2228, 629: 2228, 2228, 2228, 633: 2228, 796: 4566, 917: 4567, 1020: 4568, 1180: 4569}, + {2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 57: 2228, 146: 4322, 458: 2228, 462: 3970, 3969, 2228, 2228, 467: 2228, 475: 2228, 478: 2228, 563: 2228, 571: 2228, 573: 2228, 629: 2228, 2228, 2228, 633: 2228, 796: 4566, 918: 4567, 1021: 4568, 1181: 4569}, {146: 4324, 478: 4570}, {2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 57: 2227, 458: 2227, 462: 2227, 2227, 2227, 2227, 467: 2227, 475: 2227, 478: 2227, 563: 2227, 571: 2227, 573: 2227, 629: 2227, 2227, 2227, 633: 2227}, // 2120 @@ -8551,7 +8551,7 @@ var ( {2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 57: 2213, 458: 2213, 462: 2213, 2213, 2213, 2213, 467: 2213, 475: 2213, 478: 2213, 563: 2213, 571: 2213, 573: 2213, 629: 2213, 2213, 2213, 633: 2213}, {2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 57: 2226, 458: 2226, 462: 2226, 2226, 2226, 2226, 467: 2226, 475: 2226, 478: 2226, 563: 2226, 571: 2226, 573: 2226, 629: 2226, 2226, 2226, 633: 2226}, {2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 57: 2214, 458: 2214, 462: 2214, 2214, 2214, 2214, 467: 2214, 475: 2214, 478: 2214, 563: 2214, 571: 2214, 573: 2214, 629: 2214, 2214, 2214, 633: 2214}, - {575: 4579, 4576, 4577, 4578, 1056: 4574, 4575, 4573}, + {575: 4579, 4576, 4577, 4578, 1057: 4574, 4575, 4573}, // 2125 {2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 57: 2215, 458: 2215, 462: 2215, 2215, 2215, 2215, 467: 2215, 475: 2215, 478: 2215, 563: 2215, 571: 2215, 573: 2215, 629: 2215, 2215, 2215, 633: 2215}, {2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 57: 2164, 458: 2164, 462: 2164, 2164, 2164, 2164, 467: 2164, 475: 2164, 478: 2164, 563: 2164, 571: 2164, 573: 2164, 629: 2164, 2164, 2164, 633: 2164}, @@ -8593,22 +8593,22 @@ var ( {2330, 2330, 2330, 2330, 7: 2330, 475: 2330}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 4606}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4345, 831: 4607}, - {2332, 2332, 2332, 2332, 7: 2332, 4602, 4603, 475: 2332, 912: 4608}, + {2332, 2332, 2332, 2332, 7: 2332, 4602, 4603, 475: 2332, 913: 4608}, // 2160 {2380, 2380, 2380, 2380, 7: 2380, 475: 2380}, {2381, 2381, 2381, 2381, 7: 2381, 475: 2381}, {2382, 2382, 2382, 2382, 7: 2382, 475: 2382}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4614, 965: 4613, 1143: 4612}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4614, 966: 4613, 1144: 4612}, {2383, 2383, 2383, 2383, 7: 4616, 475: 2383}, // 2165 {1271, 1271, 1271, 1271, 7: 1271, 475: 1271}, - {1261, 1261, 1261, 1261, 7: 1261, 475: 1261, 515: 4512, 4511, 929: 4615}, + {1261, 1261, 1261, 1261, 7: 1261, 475: 1261, 515: 4512, 4511, 930: 4615}, {1269, 1269, 1269, 1269, 7: 1269, 475: 1269}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4614, 965: 4617}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4614, 966: 4617}, {1270, 1270, 1270, 1270, 7: 1270, 475: 1270}, // 2170 {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, 550, 550, 550, 550, 550, 550, 550, 58: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4621, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 638: 550, 804: 4620, 823: 4619}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 775: 4624, 818: 4622}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 774: 4624, 818: 4622}, {549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 58: 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 459: 549, 475: 549, 493: 549, 529: 549, 553: 549, 638: 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, 548, 548, 548, 548, 548, 548, 548, 58: 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 459: 548, 475: 548, 493: 548, 529: 548, 553: 548, 638: 548}, {2386, 2386, 2386, 2386, 7: 2386, 475: 2386}, @@ -8619,23 +8619,23 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4627, 2664, 2665, 2663}, {2324, 2324, 2324, 2324, 7: 2324, 30: 2324, 57: 2324, 98: 2324, 155: 2324, 460: 2324, 475: 2324, 481: 2324, 635: 2324, 638: 2324}, // 2180 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 775: 4624, 818: 4630}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 774: 4624, 818: 4630}, {2387, 2387, 2387, 2387, 7: 2387, 475: 2387}, {30: 4631}, {2389, 2389, 2389, 2389, 7: 2389, 475: 2389}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 775: 4624, 818: 4634}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 774: 4624, 818: 4634}, // 2185 {2388, 2388, 2388, 2388, 7: 2388, 475: 2388}, {30: 4635}, {2390, 2390, 2390, 2390, 7: 2390, 475: 2390}, {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, 550, 550, 550, 550, 550, 550, 550, 58: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4621, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 638: 550, 804: 4620, 823: 4637}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 775: 4624, 818: 4638}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 774: 4624, 818: 4638}, // 2190 {2391, 2391, 2391, 2391, 7: 2391, 475: 2391}, {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, 550, 550, 550, 550, 550, 550, 550, 58: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4621, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 638: 550, 804: 4620, 823: 4640}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 775: 4624, 818: 4641}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 774: 4624, 818: 4641}, {2392, 2392, 2392, 2392, 7: 2392, 475: 2392}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 775: 4624, 818: 4643}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 774: 4624, 818: 4643}, // 2195 {2393, 2393, 2393, 2393, 7: 2393, 475: 2393}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4645, 2664, 2665, 2663}, @@ -8662,19 +8662,19 @@ var ( {2361, 2361, 2361, 2361, 7: 2361, 475: 2361}, // 2215 {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 4664}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4599, 2664, 2665, 2663, 1119: 4665}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4599, 2664, 2665, 2663, 1120: 4665}, {2384, 2384, 2384, 2384, 7: 2384, 475: 2384}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4667, 2664, 2665, 2663}, {2385, 2385, 2385, 2385, 7: 2385, 475: 2385}, // 2220 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4669, 2664, 2665, 2663}, {2395, 2395, 2395, 2395, 7: 2395, 475: 2395}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 775: 4671}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 774: 4671}, {2396, 2396, 2396, 2396, 7: 4626, 475: 2396}, {2397, 2397, 2397, 2397, 7: 2397, 475: 2397}, // 2225 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4674}, - {1992, 1992, 1992, 1992, 7: 1992, 475: 1992, 657: 4677, 659: 4676, 893: 4675}, + {1992, 1992, 1992, 1992, 7: 1992, 475: 1992, 657: 4677, 659: 4676, 894: 4675}, {2398, 2398, 2398, 2398, 7: 2398, 475: 2398}, {1991, 1991, 1991, 1991, 7: 1991, 475: 1991}, {1990, 1990, 1990, 1990, 7: 1990, 475: 1990}, @@ -8682,30 +8682,30 @@ var ( {136: 4621, 493: 550, 804: 4620, 823: 4679}, {493: 2638, 722: 4680}, {2399, 2399, 2399, 2399, 7: 2399, 475: 2399}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 775: 4624, 818: 4682}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 638: 4623, 648: 4625, 2664, 2665, 2663, 774: 4624, 818: 4682}, {2400, 2400, 2400, 2400, 7: 2400, 475: 2400}, // 2235 {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 459: 1805, 552: 4701, 769: 4815}, {2406, 2406, 2406, 2406, 7: 2406, 475: 2406}, {1805, 1805, 1805, 1805, 7: 1805, 103: 1805, 136: 1805, 459: 1805, 475: 1805, 552: 4701, 769: 4769, 804: 1805}, {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 552: 4701, 769: 4760}, - {563: 4299, 571: 4693, 573: 4688, 629: 4691, 635: 4300, 662: 4692, 4689, 813: 4690, 1170: 4694}, + {563: 4299, 571: 4693, 573: 4688, 629: 4691, 635: 4300, 662: 4692, 4689, 813: 4690, 1171: 4694}, // 2240 {563: 4754}, - {2: 2336, 2336, 2336, 2336, 2336, 8: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 58: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 459: 2336, 563: 4299, 635: 4300, 813: 4710, 1044: 4748}, + {2: 2336, 2336, 2336, 2336, 2336, 8: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 58: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 459: 2336, 563: 4299, 635: 4300, 813: 4710, 1045: 4748}, {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 459: 1805, 468: 1805, 552: 4701, 769: 4742}, - {2: 2336, 2336, 2336, 2336, 2336, 8: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 58: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 459: 2336, 468: 2336, 563: 4299, 635: 4300, 813: 4710, 1044: 4711}, + {2: 2336, 2336, 2336, 2336, 2336, 8: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 58: 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 2336, 459: 2336, 468: 2336, 563: 4299, 635: 4300, 813: 4710, 1045: 4711}, {563: 4699}, // 2245 {459: 4695}, {432, 432, 432, 432, 7: 432, 57: 432, 475: 432}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 4696}, {57: 4697, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {2228, 2228, 2228, 2228, 7: 2228, 57: 2228, 146: 4322, 462: 3970, 3969, 475: 2228, 796: 4323, 917: 4567, 1020: 4698}, + {2228, 2228, 2228, 2228, 7: 2228, 57: 2228, 146: 4322, 462: 3970, 3969, 475: 2228, 796: 4323, 918: 4567, 1021: 4698}, // 2250 {2186, 2186, 2186, 2186, 7: 2186, 57: 2186, 475: 2186}, {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 459: 1805, 552: 4701, 769: 4700}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 648: 4705, 2664, 2665, 2663, 856: 4704}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 648: 4705, 2664, 2665, 2663, 857: 4704}, {462: 3970, 3969, 796: 4702}, {569: 4703}, // 2255 @@ -8715,27 +8715,27 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4507, 648: 4024, 2664, 2665, 2663, 729: 4506, 812: 4505, 822: 4707}, {7: 4516, 57: 4708}, // 2260 - {631: 4500, 892: 4709}, + {631: 4500, 893: 4709}, {2187, 2187, 2187, 2187, 7: 2187, 57: 2187, 475: 2187}, {2: 2335, 2335, 2335, 2335, 2335, 8: 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 58: 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 2335, 459: 2335, 468: 2335}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 468: 1801, 648: 4713, 2664, 2665, 2663, 856: 4714, 925: 4712}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 468: 1801, 648: 4713, 2664, 2665, 2663, 857: 4714, 926: 4712}, {459: 4722}, // 2265 {93: 4720, 459: 1800, 468: 1800}, {459: 1791, 468: 4715}, - {140: 4718, 169: 4717, 182: 4719, 886: 4716}, + {140: 4718, 169: 4717, 182: 4719, 887: 4716}, {459: 1790}, {1784, 1784, 1784, 1784, 1784, 7: 1784, 29: 1784, 57: 1784, 92: 1784, 1784, 1784, 1784, 1784, 1784, 458: 1784, 1784, 1784, 468: 1784, 475: 1784, 483: 1784}, // 2270 {1783, 1783, 1783, 1783, 1783, 7: 1783, 29: 1783, 57: 1783, 92: 1783, 1783, 1783, 1783, 1783, 1783, 458: 1783, 1783, 1783, 468: 1783, 475: 1783, 483: 1783}, {1782, 1782, 1782, 1782, 1782, 7: 1782, 29: 1782, 57: 1782, 92: 1782, 1782, 1782, 1782, 1782, 1782, 458: 1782, 1782, 1782, 468: 1782, 475: 1782, 483: 1782}, - {140: 4718, 169: 4717, 182: 4719, 886: 4721}, + {140: 4718, 169: 4717, 182: 4719, 887: 4721}, {459: 1789}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4507, 648: 4024, 2664, 2665, 2663, 729: 4506, 812: 4505, 822: 4723}, // 2275 {7: 4516, 57: 4724}, - {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 858: 4725}, - {2188, 2188, 2188, 2188, 4730, 7: 2188, 29: 4727, 57: 2188, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2188, 839: 4731, 841: 4728, 849: 4732, 857: 4726}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 859: 4725}, + {2188, 2188, 2188, 2188, 4730, 7: 2188, 29: 4727, 57: 2188, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2188, 839: 4731, 841: 4728, 849: 4732, 858: 4726}, {1798, 1798, 1798, 1798, 1798, 7: 1798, 29: 1798, 57: 1798, 92: 1798, 1798, 1798, 1798, 1798, 1798, 460: 1798, 468: 1798, 475: 1798, 483: 1798}, {482: 4171, 493: 1987, 723: 4740}, // 2280 @@ -8745,8 +8745,8 @@ var ( {1793, 1793, 1793, 1793, 1793, 7: 1793, 29: 1793, 57: 1793, 92: 1793, 1793, 1793, 1793, 1793, 1793, 460: 1793, 468: 1793, 475: 1793, 483: 1793}, {1792, 1792, 1792, 1792, 1792, 7: 1792, 29: 1792, 57: 1792, 92: 1792, 1792, 1792, 1792, 1792, 1792, 460: 1792, 468: 1792, 475: 1792, 483: 1792}, // 2285 - {140: 4718, 169: 4717, 182: 4719, 886: 4736}, - {140: 4718, 169: 4717, 182: 4719, 886: 4735}, + {140: 4718, 169: 4717, 182: 4719, 887: 4736}, + {140: 4718, 169: 4717, 182: 4719, 887: 4735}, {1785, 1785, 1785, 1785, 1785, 7: 1785, 29: 1785, 57: 1785, 92: 1785, 1785, 1785, 1785, 1785, 1785, 458: 1785, 460: 1785, 468: 1785, 475: 1785, 483: 1785}, {1786, 1786, 1786, 1786, 1786, 7: 1786, 29: 1786, 57: 1786, 92: 1786, 1786, 1786, 1786, 1786, 1786, 458: 1786, 460: 1786, 468: 1786, 475: 1786, 483: 1786}, {1794, 1794, 1794, 1794, 1794, 7: 1794, 29: 1794, 57: 1794, 92: 1794, 1794, 1794, 1794, 1794, 1794, 460: 1794, 468: 1794, 475: 1794, 483: 1794}, @@ -8755,51 +8755,51 @@ var ( {1795, 1795, 1795, 1795, 1795, 7: 1795, 29: 1795, 57: 1795, 92: 1795, 1795, 1795, 1795, 1795, 1795, 460: 1795, 468: 1795, 475: 1795, 483: 1795}, {493: 2638, 722: 2637, 730: 4741}, {1797, 1797, 1797, 1797, 1797, 7: 1797, 29: 1797, 57: 1797, 92: 1797, 1797, 1797, 1797, 1797, 1797, 460: 1797, 468: 1797, 475: 1797, 483: 1797}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 468: 1801, 648: 4713, 2664, 2665, 2663, 856: 4714, 925: 4743}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 468: 1801, 648: 4713, 2664, 2665, 2663, 857: 4714, 926: 4743}, // 2295 {459: 4744}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4507, 648: 4024, 2664, 2665, 2663, 729: 4506, 812: 4505, 822: 4745}, {7: 4516, 57: 4746}, - {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 858: 4747}, - {2189, 2189, 2189, 2189, 4730, 7: 2189, 29: 4727, 57: 2189, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2189, 839: 4731, 841: 4728, 849: 4732, 857: 4726}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 859: 4747}, + {2189, 2189, 2189, 2189, 4730, 7: 2189, 29: 4727, 57: 2189, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2189, 839: 4731, 841: 4728, 849: 4732, 858: 4726}, // 2300 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 648: 4705, 2664, 2665, 2663, 856: 4749}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 648: 4705, 2664, 2665, 2663, 857: 4749}, {459: 4750}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4507, 648: 4024, 2664, 2665, 2663, 729: 4506, 812: 4505, 822: 4751}, {7: 4516, 57: 4752}, - {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 858: 4753}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 859: 4753}, // 2305 - {2190, 2190, 2190, 2190, 4730, 7: 2190, 29: 4727, 57: 2190, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2190, 839: 4731, 841: 4728, 849: 4732, 857: 4726}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 468: 1801, 648: 4713, 2664, 2665, 2663, 856: 4714, 925: 4755}, + {2190, 2190, 2190, 2190, 4730, 7: 2190, 29: 4727, 57: 2190, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2190, 839: 4731, 841: 4728, 849: 4732, 858: 4726}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 1801, 468: 1801, 648: 4713, 2664, 2665, 2663, 857: 4714, 926: 4755}, {459: 4756}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4507, 648: 4024, 2664, 2665, 2663, 729: 4506, 812: 4505, 822: 4757}, {7: 4516, 57: 4758}, // 2310 - {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 858: 4759}, - {2191, 2191, 2191, 2191, 4730, 7: 2191, 29: 4727, 57: 2191, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2191, 839: 4731, 841: 4728, 849: 4732, 857: 4726}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 859: 4759}, + {2191, 2191, 2191, 2191, 4730, 7: 2191, 29: 4727, 57: 2191, 93: 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 475: 2191, 839: 4731, 841: 4728, 849: 4732, 858: 4726}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4761, 2664, 2665, 2663}, - {220: 4763, 229: 4765, 232: 4764, 1115: 4762}, + {220: 4763, 229: 4765, 232: 4764, 1116: 4762}, {459: 4766}, // 2315 {57: 2146, 459: 2146}, {57: 2145, 459: 2145}, {57: 2144, 459: 2144}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 4767}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 4767}, {7: 4027, 57: 4768}, // 2320 {2403, 2403, 2403, 2403, 7: 2403, 475: 2403}, {550, 550, 550, 550, 7: 550, 103: 550, 136: 4621, 459: 550, 475: 550, 804: 4620, 823: 4770}, - {2082, 2082, 2082, 2082, 7: 2082, 103: 4772, 459: 4773, 475: 2082, 1075: 4771}, + {2082, 2082, 2082, 2082, 7: 2082, 103: 4772, 459: 4773, 475: 2082, 1076: 4771}, {2405, 2405, 2405, 2405, 7: 2405, 475: 2405}, {493: 2638, 722: 4814}, // 2325 - {475: 4776, 932: 4775, 1074: 4774}, + {475: 4776, 933: 4775, 1075: 4774}, {7: 4812, 57: 4811}, {7: 2080, 57: 2080}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4777, 2664, 2665, 2663}, {4: 2059, 2059, 7: 2059, 15: 2059, 2059, 2059, 2059, 2059, 2059, 2059, 2059, 2059, 2059, 2059, 2059, 30: 2059, 2059, 2059, 2059, 2059, 2059, 2059, 57: 2059, 145: 4782, 327: 4781, 459: 2059, 464: 4780, 484: 4779, 635: 2059, 1246: 4778}, // 2330 - {4: 2072, 2072, 7: 2072, 15: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 30: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 57: 2072, 459: 2072, 635: 2072, 931: 4798}, + {4: 2072, 2072, 7: 2072, 15: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 30: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 57: 2072, 459: 2072, 635: 2072, 932: 4798}, {337: 4783, 530: 4784}, {4: 2056, 2056, 7: 2056, 15: 2056, 2056, 2056, 2056, 2056, 2056, 2056, 2056, 2056, 2056, 2056, 2056, 30: 2056, 2056, 2056, 2056, 2056, 2056, 2056, 57: 2056, 459: 2056, 635: 2056}, {4: 2054, 2054, 7: 2054, 15: 2054, 2054, 2054, 2054, 2054, 2054, 2054, 2054, 2054, 2054, 2054, 2054, 30: 2054, 2054, 2054, 2054, 2054, 2054, 2054, 57: 2054, 459: 2054, 635: 2054}, @@ -8807,45 +8807,45 @@ var ( // 2335 {381: 4793}, {459: 4785}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 644: 4787, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4788, 928: 4789, 1055: 4786}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 644: 4787, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4788, 929: 4789, 1056: 4786}, {7: 4791, 57: 4790}, {7: 1881, 57: 1881}, // 2340 {7: 1880, 57: 1880, 471: 3573, 3572, 3578, 509: 3574, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, {7: 1868, 57: 1868}, {4: 2055, 2055, 7: 2055, 15: 2055, 2055, 2055, 2055, 2055, 2055, 2055, 2055, 2055, 2055, 2055, 2055, 30: 2055, 2055, 2055, 2055, 2055, 2055, 2055, 57: 2055, 459: 2055, 635: 2055}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 644: 4787, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4788, 928: 4792}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 644: 4787, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4788, 929: 4792}, {7: 1867, 57: 1867}, // 2345 {459: 4795, 644: 4794}, {4: 2058, 2058, 7: 2058, 15: 2058, 2058, 2058, 2058, 2058, 2058, 2058, 2058, 2058, 2058, 2058, 2058, 30: 2058, 2058, 2058, 2058, 2058, 2058, 2058, 57: 2058, 459: 2058, 635: 2058}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 644: 4787, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4788, 928: 4789, 1055: 4796}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 644: 4787, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4788, 929: 4789, 1056: 4796}, {7: 4791, 57: 4797}, {4: 2057, 2057, 7: 2057, 15: 2057, 2057, 2057, 2057, 2057, 2057, 2057, 2057, 2057, 2057, 2057, 2057, 30: 2057, 2057, 2057, 2057, 2057, 2057, 2057, 57: 2057, 459: 2057, 635: 2057}, // 2350 {4: 4131, 4802, 7: 2077, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 30: 4139, 4132, 4135, 4134, 4137, 4138, 4140, 57: 2077, 459: 4800, 635: 4136, 758: 4087, 762: 4088, 764: 4141, 798: 4801, 1287: 4799}, {7: 2078, 57: 2078}, - {100: 4805, 1117: 4804, 1286: 4803}, + {100: 4805, 1118: 4804, 1286: 4803}, {2071, 2071, 4: 2071, 2071, 7: 2071, 15: 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 30: 2071, 2071, 2071, 2071, 2071, 2071, 2071, 57: 2071, 459: 2071, 635: 2071}, {31: 4270}, // 2355 {7: 4809, 57: 4808}, {7: 2075, 57: 2075}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4806, 2664, 2665, 2663}, - {4: 2072, 2072, 7: 2072, 15: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 30: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 57: 2072, 635: 2072, 931: 4807}, + {4: 2072, 2072, 7: 2072, 15: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 30: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 57: 2072, 635: 2072, 932: 4807}, {4: 4131, 4802, 7: 2073, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 30: 4139, 4132, 4135, 4134, 4137, 4138, 4140, 57: 2073, 635: 4136, 758: 4087, 762: 4088, 764: 4141, 798: 4801}, // 2360 {7: 2076, 57: 2076}, - {100: 4805, 1117: 4810}, + {100: 4805, 1118: 4810}, {7: 2074, 57: 2074}, {2081, 2081, 2081, 2081, 7: 2081, 458: 2081, 2081, 2081, 465: 2081, 474: 2081, 2081, 484: 2081, 492: 2081, 553: 2081, 632: 2081}, - {475: 4776, 932: 4813}, + {475: 4776, 933: 4813}, // 2365 {7: 2079, 57: 2079}, {2404, 2404, 2404, 2404, 7: 2404, 475: 2404}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4817, 648: 4024, 2664, 2665, 2663, 729: 4345, 831: 4816}, - {2332, 2332, 2332, 2332, 7: 2332, 4602, 4603, 475: 2332, 912: 4825}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 648: 4024, 2664, 2665, 2663, 662: 2323, 2323, 729: 4345, 819: 4687, 831: 4819, 884: 4820, 948: 4821, 1120: 4818}, + {2332, 2332, 2332, 2332, 7: 2332, 4602, 4603, 475: 2332, 913: 4825}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 648: 4024, 2664, 2665, 2663, 662: 2323, 2323, 729: 4345, 819: 4687, 831: 4819, 885: 4820, 949: 4821, 1121: 4818}, // 2370 {7: 4823, 57: 4822}, {7: 429, 57: 429}, @@ -8853,14 +8853,14 @@ var ( {7: 427, 57: 427}, {2407, 2407, 2407, 2407, 7: 2407, 475: 2407}, // 2375 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 648: 4024, 2664, 2665, 2663, 662: 2323, 2323, 729: 4345, 819: 4687, 831: 4819, 884: 4820, 948: 4824}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 648: 4024, 2664, 2665, 2663, 662: 2323, 2323, 729: 4345, 819: 4687, 831: 4819, 885: 4820, 949: 4824}, {7: 426, 57: 426}, {2408, 2408, 2408, 2408, 7: 2408, 475: 2408}, {13: 3720, 486: 3721, 634: 3719, 759: 4827}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 464: 4829, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 4828}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 464: 4829, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 4828}, // 2380 - {257, 257, 257, 257, 7: 257, 467: 4831, 475: 257, 1066: 4833}, - {257, 257, 257, 257, 7: 257, 467: 4831, 475: 257, 1066: 4830}, + {257, 257, 257, 257, 7: 257, 467: 4831, 475: 257, 1067: 4833}, + {257, 257, 257, 257, 7: 257, 467: 4831, 475: 257, 1067: 4830}, {2409, 2409, 2409, 2409, 7: 2409, 475: 2409}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3375, 648: 3377, 2664, 2665, 2663, 725: 3374, 852: 4832}, {256, 256, 256, 256, 7: 256, 475: 256}, @@ -8880,7 +8880,7 @@ var ( {254, 254, 254, 254, 7: 254, 475: 254}, {6: 388, 38: 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, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 458: 382, 382, 382, 464: 382, 382, 382, 382, 474: 382, 382, 484: 382, 382, 382, 492: 382, 553: 382, 632: 382, 634: 382, 382}, - {4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 464: 4130, 466: 4167, 2106, 485: 4844, 2106, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 780: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4847}, + {4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 464: 4130, 466: 4167, 2106, 485: 4844, 2106, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4847}, {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, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 458: 381, 381, 381, 464: 381, 381, 381, 381, 474: 381, 381, 484: 381, 381, 381, 492: 381, 553: 381, 632: 381, 634: 381, 381}, // 2400 {461: 4850, 464: 4849}, @@ -8891,7 +8891,7 @@ var ( // 2405 {2422, 2422, 2422, 2422, 7: 2422, 475: 2422}, {2: 1987, 1987, 1987, 1987, 1987, 8: 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 58: 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 461: 1987, 464: 1987, 482: 4171, 498: 4856, 723: 4855}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 4858, 464: 4860, 648: 4861, 2664, 2665, 2663, 864: 4859}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 4858, 464: 4860, 648: 4861, 2664, 2665, 2663, 865: 4859}, {464: 4857}, {2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 13: 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 2424, 458: 2424, 2424, 2424, 464: 2424, 2424, 2424, 2424, 474: 2424, 2424, 484: 2424, 2424, 2424, 492: 2424, 553: 2424, 632: 2424, 634: 2424, 2424}, // 2410 @@ -8926,12 +8926,12 @@ var ( {461: 4883}, // 2435 {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, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 458: 2440, 2440, 2440, 464: 2440, 2440, 2440, 2440, 474: 2440, 2440, 484: 2440, 2440, 2440, 492: 2440, 553: 2440, 632: 2440, 634: 2440, 2440}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 775: 4885}, - {2284, 2284, 7: 4626, 460: 4888, 635: 4887, 789: 4886}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 774: 4885}, + {2284, 2284, 7: 4626, 460: 4888, 635: 4887, 788: 4886}, {2445, 2445}, {873, 873, 2900, 2748, 2784, 2902, 2675, 873, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 460: 873, 573: 4905, 648: 4904, 2664, 2665, 2663, 840: 4903}, // 2440 - {493: 4893, 559: 3311, 3310, 722: 4891, 805: 4892, 971: 4890, 1148: 4889}, + {493: 4893, 559: 3311, 3310, 722: 4891, 805: 4892, 972: 4890, 1149: 4889}, {2283, 2283, 7: 4901}, {2282, 2282, 7: 2282}, {219: 4895, 223: 4897, 268: 4898, 287: 4896}, @@ -8946,10 +8946,10 @@ var ( {2276, 2276, 7: 2276}, {2278, 2278, 7: 2278}, {2277, 2277, 7: 2277}, - {493: 4893, 559: 3311, 3310, 722: 4891, 805: 4892, 971: 4902}, + {493: 4893, 559: 3311, 3310, 722: 4891, 805: 4892, 972: 4902}, {2281, 2281, 7: 2281}, // 2455 - {2284, 2284, 7: 4907, 460: 4888, 789: 4906}, + {2284, 2284, 7: 4907, 460: 4888, 788: 4906}, {872, 872, 7: 872, 57: 872, 460: 872}, {870, 870, 7: 870, 57: 870, 460: 870}, {2444, 2444}, @@ -8963,8 +8963,8 @@ var ( // 2465 {475: 4969}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 646: 4916, 648: 4915, 2664, 2665, 2663}, - {2072, 2072, 4: 2072, 2072, 15: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 30: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 190: 4090, 635: 2072, 909: 4967, 931: 4968}, - {140: 2090, 339: 4921, 377: 4922, 513: 4920, 563: 2090, 1049: 4923, 4918, 1118: 4919, 1248: 4917}, + {2072, 2072, 4: 2072, 2072, 15: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 30: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 190: 4090, 635: 2072, 910: 4967, 932: 4968}, + {140: 2090, 339: 4921, 377: 4922, 513: 4920, 563: 2090, 1050: 4923, 4918, 1119: 4919, 1248: 4917}, {2084, 2084, 100: 2084, 103: 4957, 458: 2084, 2084, 2084, 465: 2084, 474: 2084, 484: 2084, 492: 2084, 553: 2084, 632: 2084, 1249: 4956}, // 2470 {140: 4944, 563: 4943}, @@ -8977,12 +8977,12 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 4927}, {493: 2638, 722: 2637, 730: 4926}, {2092, 2092, 100: 2092, 103: 2092, 458: 2092, 2092, 2092, 465: 2092, 474: 2092, 484: 2092, 492: 2092, 553: 2092, 632: 2092}, - {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 793: 3330, 816: 4928}, + {105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 491: 3234, 494: 3232, 3233, 3231, 3229, 517: 3340, 3337, 3339, 3338, 3334, 3336, 3335, 3332, 3333, 3331, 3341, 720: 3230, 3228, 792: 3330, 816: 4928}, // 2480 {2093, 2093, 100: 2093, 103: 2093, 458: 2093, 2093, 2093, 465: 2093, 474: 2093, 484: 2093, 492: 2093, 553: 2093, 632: 2093}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4934}, {459: 4931}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 4932}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 4932}, {7: 4027, 57: 4933}, // 2485 {2094, 2094, 100: 2094, 103: 2094, 458: 2094, 2094, 2094, 465: 2094, 474: 2094, 484: 2094, 492: 2094, 553: 2094, 632: 2094}, @@ -8991,7 +8991,7 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 4941}, {459: 4938}, // 2490 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 4939}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 4939}, {7: 4027, 57: 4940}, {2096, 2096, 100: 2096, 103: 2096, 458: 2096, 2096, 2096, 465: 2096, 474: 2096, 484: 2096, 492: 2096, 553: 2096, 632: 2096}, {57: 4942, 471: 3573, 3572, 3578, 509: 3574, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, @@ -9007,7 +9007,7 @@ var ( {482: 4950}, {493: 2638, 722: 4951}, {459: 2099}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 2250, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 4953, 981: 4954}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 2250, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 4953, 982: 4954}, // 2505 {7: 4027, 57: 2249}, {57: 4955}, @@ -9016,9 +9016,9 @@ var ( {493: 2638, 722: 2637, 730: 4958}, // 2510 {2083, 2083, 100: 2083, 458: 2083, 2083, 2083, 465: 2083, 474: 2083, 484: 2083, 492: 2083, 553: 2083, 632: 2083}, - {2082, 2082, 458: 2082, 4773, 2082, 465: 2082, 474: 2082, 484: 2082, 492: 2082, 553: 2082, 632: 2082, 1075: 4966}, + {2082, 2082, 458: 2082, 4773, 2082, 465: 2082, 474: 2082, 484: 2082, 492: 2082, 553: 2082, 632: 2082, 1076: 4966}, {646: 4961}, - {140: 2090, 563: 2090, 1049: 4923, 4918, 1118: 4962}, + {140: 2090, 563: 2090, 1050: 4923, 4918, 1119: 4962}, {2086, 2086, 210: 4964, 458: 2086, 2086, 2086, 465: 2086, 474: 2086, 484: 2086, 492: 2086, 553: 2086, 632: 2086, 1288: 4963}, // 2515 {2087, 2087, 458: 2087, 2087, 2087, 465: 2087, 474: 2087, 484: 2087, 492: 2087, 553: 2087, 632: 2087}, @@ -9029,26 +9029,26 @@ var ( // 2520 {2415, 2415, 4: 4131, 4802, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 30: 4139, 4132, 4135, 4134, 4137, 4138, 4140, 635: 4136, 758: 4087, 762: 4088, 764: 4141, 798: 4801}, {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, 550, 550, 550, 550, 550, 550, 550, 58: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4621, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 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: 4620, 823: 4970}, - {2357, 2357, 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 775: 4972, 1257: 4971}, + {2357, 2357, 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 774: 4972, 1257: 4971}, {2417, 2417}, {7: 4626, 481: 4973}, // 2525 {459: 4974}, - {475: 4776, 932: 4775, 1074: 4975}, + {475: 4776, 933: 4775, 1075: 4975}, {7: 4812, 57: 4976}, {2356, 2356}, {2418, 2418}, // 2530 - {136: 4979, 859: 96, 1053: 4980}, - {859: 95}, - {859: 4981}, + {136: 4979, 860: 96, 1054: 4980}, + {860: 95}, + {860: 4981}, {461: 4982}, {17, 17, 175: 17, 341: 4984, 645: 17, 1227: 4983}, // 2535 {15, 15, 175: 4987, 645: 15, 1226: 4986}, {493: 2638, 722: 4985}, {16, 16, 175: 16, 645: 16}, - {81, 81, 645: 3849, 927: 4994}, + {81, 81, 645: 3849, 928: 4994}, {13, 13, 179: 13, 354: 4989, 645: 13, 1251: 4988}, // 2540 {11, 11, 179: 4992, 645: 11, 1250: 4991}, @@ -9066,7 +9066,7 @@ var ( {493: 2638, 722: 4999}, {57, 57}, {58, 58}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5005, 1121: 5006, 1291: 5004}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5005, 1122: 5006, 1291: 5004}, {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, 68, 68, 68, 68, 68, 68, 68, 58: 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68}, // 2555 {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, 67, 67, 67, 67, 67, 67, 67, 58: 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, @@ -9079,48 +9079,48 @@ var ( {63, 63, 7: 63, 136: 5010}, {62, 62, 7: 62}, {64, 64, 7: 64}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5005, 1121: 5013}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5005, 1122: 5013}, // 2565 {60, 60, 7: 60}, {70, 70}, - {136: 4979, 859: 96, 1053: 5018}, + {136: 4979, 860: 96, 1054: 5018}, {461: 5017}, {54, 54}, // 2570 - {859: 5019}, + {860: 5019}, {461: 5020}, - {474: 5021, 481: 2052, 492: 5022, 1017: 5023}, + {474: 5021, 481: 2052, 492: 5022, 1018: 5023}, {2051, 2051, 458: 2051, 2051, 2051, 465: 2051, 481: 2051, 484: 2051, 553: 2051, 632: 2051}, {2050, 2050, 458: 2050, 2050, 2050, 465: 2050, 481: 2050, 484: 2050, 553: 2050, 632: 2050}, // 2575 {481: 5024}, {553: 5025}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5026}, - {98, 98, 98: 98, 107: 98, 459: 98, 474: 98, 498: 98, 634: 5028, 645: 98, 1160: 5027}, - {94, 94, 98: 3828, 107: 3827, 459: 94, 474: 94, 498: 94, 645: 94, 820: 3826, 1027: 5031}, + {98, 98, 98: 98, 107: 98, 459: 98, 474: 98, 498: 98, 634: 5028, 645: 98, 1161: 5027}, + {94, 94, 98: 3828, 107: 3827, 459: 94, 474: 94, 498: 94, 645: 94, 820: 3826, 1028: 5031}, // 2580 {498: 5029}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 5030}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 5030}, {97, 97, 98: 97, 107: 97, 459: 97, 474: 97, 498: 97, 645: 97}, - {81, 81, 459: 81, 474: 81, 498: 81, 645: 3849, 927: 5032}, + {81, 81, 459: 81, 474: 81, 498: 81, 645: 3849, 928: 5032}, {100, 100, 459: 100, 474: 5034, 498: 100, 1203: 5033}, // 2585 - {2238, 2238, 459: 5037, 498: 2238, 1166: 5038}, + {2238, 2238, 459: 5037, 498: 2238, 1167: 5038}, {493: 2638, 722: 5035}, {645: 5036}, {99, 99, 459: 99, 498: 99}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 2244, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 554: 3237, 648: 4024, 2664, 2665, 2663, 697: 5051, 729: 5050, 982: 5049, 1164: 5048, 5052}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 2244, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 554: 3237, 648: 4024, 2664, 2665, 2663, 697: 5051, 729: 5050, 983: 5049, 1165: 5048, 5052}, // 2590 {75, 75, 498: 5040, 1220: 5039}, {101, 101}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3664, 2664, 2665, 2663, 698: 5043, 1051: 5042, 1219: 5041}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3664, 2664, 2665, 2663, 698: 5043, 1052: 5042, 1219: 5041}, {74, 74, 7: 5046}, {72, 72, 7: 72}, // 2595 {482: 5044}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5045}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5045}, {71, 71, 7: 71}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3664, 2664, 2665, 2663, 698: 5043, 1051: 5047}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3664, 2664, 2665, 2663, 698: 5043, 1052: 5047}, {73, 73, 7: 73}, // 2600 {7: 5054, 57: 2243}, @@ -9130,7 +9130,7 @@ var ( {57: 5053}, // 2605 {2237, 2237, 498: 2237}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 554: 3237, 648: 4024, 2664, 2665, 2663, 697: 5051, 729: 5050, 982: 5055}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 554: 3237, 648: 4024, 2664, 2665, 2663, 697: 5051, 729: 5050, 983: 5055}, {7: 2241, 57: 2241}, {7: 158, 161: 158, 458: 158, 487: 158, 554: 1779, 637: 158, 654: 1779}, {7: 123, 458: 123, 123, 487: 123, 554: 1750, 637: 123, 654: 1750}, @@ -9167,9 +9167,9 @@ var ( // 2635 {7: 136, 458: 136, 136, 487: 136, 637: 136}, {7: 135, 458: 135, 135, 487: 135, 637: 135}, - {99: 5094, 1000: 5093}, + {99: 5094, 1001: 5093}, {7: 131, 458: 131, 131, 487: 131, 637: 131}, - {888: 5092}, + {889: 5092}, // 2640 {7: 129, 458: 129, 129, 487: 129, 637: 129}, {7: 126, 458: 126, 126, 487: 126, 637: 126}, @@ -9191,15 +9191,15 @@ var ( // 2655 {7: 115, 458: 115, 115, 487: 115, 637: 115}, {7: 146, 458: 146, 146, 487: 146, 637: 146}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 5106}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 5106}, {7: 4027, 57: 5107}, {7: 148, 458: 148, 487: 148, 637: 148}, // 2660 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5056, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 5058, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 5064, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 5060, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5057, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 5065, 3095, 2831, 3050, 5059, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 5062, 2744, 2745, 2981, 5063, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 5061, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5067, 483: 5090, 555: 5084, 631: 5088, 5073, 635: 5083, 638: 5077, 641: 5086, 648: 3377, 2664, 2665, 2663, 653: 5078, 656: 5082, 661: 5079, 725: 5066, 731: 5081, 786: 5068, 814: 5072, 837: 5087, 846: 5085, 919: 5069, 937: 5070, 5076, 943: 5071, 5155, 952: 5080, 954: 5089}, - {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, 112, 112, 112, 112, 112, 112, 112, 58: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5122, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 509: 112, 553: 5121, 939: 5123, 1060: 5124}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5056, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 5058, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 5064, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 5060, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5057, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 5065, 3095, 2831, 3050, 5059, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 5062, 2744, 2745, 2981, 5063, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 5061, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5067, 483: 5090, 555: 5084, 631: 5088, 5073, 635: 5083, 638: 5077, 641: 5086, 648: 3377, 2664, 2665, 2663, 653: 5078, 656: 5082, 661: 5079, 725: 5066, 731: 5081, 785: 5068, 814: 5072, 837: 5087, 846: 5085, 920: 5069, 938: 5070, 5076, 944: 5071, 5155, 953: 5080, 955: 5089}, + {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, 112, 112, 112, 112, 112, 112, 112, 58: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5122, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 509: 112, 553: 5121, 940: 5123, 1061: 5124}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5113, 848: 5114}, {714, 714, 7: 714, 14: 714, 58: 714, 99: 714, 141: 714, 460: 714, 468: 714, 482: 714, 554: 5119, 637: 714, 652: 714, 654: 5118, 714}, - {1168, 1168, 7: 1168, 14: 1168, 58: 1168, 99: 1168, 141: 1168, 459: 3654, 1168, 468: 1168, 482: 1168, 637: 1168, 652: 1168, 655: 1168, 1069: 5117}, + {1168, 1168, 7: 1168, 14: 1168, 58: 1168, 99: 1168, 141: 1168, 459: 3654, 1168, 468: 1168, 482: 1168, 637: 1168, 652: 1168, 655: 1168, 1070: 5117}, // 2665 {710, 710, 7: 710, 460: 710}, {102, 102, 7: 5115}, @@ -9214,27 +9214,27 @@ var ( {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, 110, 110, 110, 110, 110, 110, 110, 58: 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 509: 110}, // 2675 {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, 109, 109, 109, 109, 109, 109, 109, 58: 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 509: 109}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 5125, 648: 5126, 2664, 2665, 2663, 1083: 5127}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 5125, 648: 5126, 2664, 2665, 2663, 1084: 5127}, {487: 108, 637: 108, 639: 5153}, {487: 104, 637: 104, 639: 5150}, {487: 5128}, // 2680 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 869: 5130, 905: 5131}, - {190, 190, 7: 190, 14: 190, 58: 190, 141: 5135, 460: 190, 652: 190, 1152: 5134}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 870: 5130, 906: 5131}, + {190, 190, 7: 190, 14: 190, 58: 190, 141: 5135, 460: 190, 652: 190, 1153: 5134}, {225, 225, 7: 225, 14: 225, 58: 225, 460: 225, 652: 225}, {103, 103, 7: 5132}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 869: 5133}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 870: 5133}, // 2685 {224, 224, 7: 224, 14: 224, 58: 224, 460: 224, 652: 224}, {226, 226, 7: 226, 14: 226, 58: 226, 460: 226, 652: 226}, {460: 5137, 646: 5136}, - {14: 5148, 461: 5145, 871: 5147}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 648: 3377, 2664, 2665, 2663, 725: 5139, 1153: 5138}, + {14: 5148, 461: 5145, 872: 5147}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 648: 3377, 2664, 2665, 2663, 725: 5139, 1154: 5138}, // 2690 {188, 188, 7: 188, 14: 188, 58: 188, 460: 188, 465: 5141, 646: 5140, 652: 188}, {184, 184, 7: 184, 14: 184, 58: 184, 460: 184, 465: 184, 646: 184, 652: 184}, - {461: 5145, 871: 5146}, - {461: 5143, 562: 5144, 1036: 5142}, + {461: 5145, 872: 5146}, + {461: 5143, 562: 5144, 1037: 5142}, {186, 186, 7: 186, 14: 186, 58: 186, 460: 186, 652: 186}, // 2695 {183, 183, 7: 183, 14: 183, 58: 183, 460: 183, 652: 183}, @@ -9243,7 +9243,7 @@ var ( {187, 187, 7: 187, 14: 187, 58: 187, 460: 187, 652: 187}, {189, 189, 7: 189, 14: 189, 58: 189, 460: 189, 652: 189}, // 2700 - {461: 5143, 562: 5144, 1036: 5149}, + {461: 5143, 562: 5144, 1037: 5149}, {185, 185, 7: 185, 14: 185, 58: 185, 460: 185, 652: 185}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 5151, 648: 5152, 2664, 2665, 2663}, {487: 106, 637: 106}, @@ -9267,21 +9267,21 @@ var ( {458: 5206, 554: 1697, 654: 1697}, {7: 5108, 458: 5168, 637: 5169}, // 2720 - {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, 112, 112, 112, 112, 112, 112, 112, 58: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5122, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 509: 112, 553: 5121, 939: 5123, 1060: 5171}, + {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, 112, 112, 112, 112, 112, 112, 112, 58: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5122, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 509: 112, 553: 5121, 940: 5123, 1061: 5171}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5113, 848: 5170}, {165, 165, 7: 5115}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 5125, 648: 5126, 2664, 2665, 2663, 1083: 5172}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 5125, 648: 5126, 2664, 2665, 2663, 1084: 5172}, {637: 5173}, // 2725 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 869: 5130, 905: 5174}, - {215, 215, 7: 5132, 460: 215, 652: 5176, 940: 5175, 5177}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 870: 5130, 906: 5174}, + {215, 215, 7: 5132, 460: 215, 652: 5176, 941: 5175, 5177}, {214, 214, 14: 214, 58: 214, 460: 214}, - {131: 5197, 133: 5195, 5198, 5196, 346: 5190, 393: 5192, 942: 5194, 1258: 5193, 1276: 5191}, - {164, 164, 460: 5179, 1139: 5178}, + {131: 5197, 133: 5195, 5198, 5196, 346: 5190, 393: 5192, 943: 5194, 1258: 5193, 1276: 5191}, + {164, 164, 460: 5179, 1140: 5178}, // 2730 {167, 167}, {126: 5183, 5181, 5182, 5184, 837: 5180}, - {888: 5189}, + {889: 5189}, {493: 2638, 722: 5188}, {493: 2638, 722: 5187}, // 2735 @@ -9297,7 +9297,7 @@ var ( {212, 212, 14: 212, 58: 212, 460: 212}, {211, 211, 14: 211, 58: 211, 460: 211}, // 2745 - {210, 210, 14: 210, 58: 210, 131: 5197, 133: 5195, 5198, 5196, 460: 210, 491: 5203, 942: 5204}, + {210, 210, 14: 210, 58: 210, 131: 5197, 133: 5195, 5198, 5196, 460: 210, 491: 5203, 943: 5204}, {209, 209, 14: 209, 58: 209, 131: 209, 133: 209, 209, 209, 460: 209, 491: 209}, {461: 5202}, {461: 5201}, @@ -9309,28 +9309,28 @@ var ( {205, 205, 14: 205, 58: 205, 131: 205, 133: 205, 205, 205, 460: 205, 491: 205}, {206, 206, 14: 206, 58: 206, 131: 206, 133: 206, 206, 206, 460: 206, 491: 206}, // 2755 - {131: 5197, 133: 5195, 5198, 5196, 942: 5205}, + {131: 5197, 133: 5195, 5198, 5196, 943: 5205}, {207, 207, 14: 207, 58: 207, 131: 207, 133: 207, 207, 207, 460: 207, 491: 207}, {208, 208, 14: 208, 58: 208, 131: 208, 133: 208, 208, 208, 460: 208, 491: 208}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5207}, {637: 5208}, // 2760 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5113, 848: 5209}, - {164, 164, 7: 5115, 460: 5179, 1139: 5210}, + {164, 164, 7: 5115, 460: 5179, 1140: 5210}, {166, 166}, {2122, 2122, 7: 2122, 13: 2122, 15: 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 28: 2122, 464: 2122, 467: 2122, 486: 2122, 2122, 489: 2122, 507: 2122, 634: 2122, 637: 2122}, {239, 239}, // 2765 {2: 819, 819, 819, 819, 819, 8: 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 58: 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 459: 819, 461: 819, 819, 819, 819, 469: 819, 819, 819, 819, 819, 819, 478: 819, 481: 819, 484: 819, 486: 819, 819, 492: 819, 819, 500: 819, 509: 819, 529: 819, 552: 819, 554: 819, 819, 819, 819, 819, 819, 819, 819, 819, 564: 819, 819, 819, 819, 819, 819, 572: 819, 574: 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, 636: 819, 638: 819, 732: 819, 819, 735: 819, 819, 819, 743: 819, 755: 819, 819, 819}, {2: 817, 817, 817, 817, 817, 8: 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 58: 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 817, 459: 817, 474: 817, 481: 817, 487: 817, 565: 817, 735: 817, 817, 817}, - {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 459: 1020, 474: 1020, 565: 1020, 735: 5218, 5217, 5216, 824: 5219, 865: 5220}, + {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 459: 1020, 474: 1020, 565: 1020, 735: 5218, 5217, 5216, 824: 5219, 866: 5220}, {2: 1023, 1023, 1023, 1023, 1023, 8: 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 58: 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 459: 1023, 461: 1023, 1023, 1023, 1023, 469: 1023, 1023, 1023, 1023, 1023, 1023, 478: 1023, 481: 1023, 484: 1023, 486: 1023, 1023, 492: 1023, 1023, 500: 1023, 509: 1023, 529: 1023, 552: 1023, 554: 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 564: 1023, 1023, 1023, 1023, 1023, 1023, 572: 1023, 574: 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 1023, 636: 1023, 638: 1023, 732: 1023, 1023, 735: 1023, 1023, 1023, 743: 1023, 755: 1023, 1023, 1023}, {2: 1022, 1022, 1022, 1022, 1022, 8: 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 58: 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 459: 1022, 461: 1022, 1022, 1022, 1022, 469: 1022, 1022, 1022, 1022, 1022, 1022, 478: 1022, 481: 1022, 484: 1022, 486: 1022, 1022, 492: 1022, 1022, 500: 1022, 509: 1022, 529: 1022, 552: 1022, 554: 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 564: 1022, 1022, 1022, 1022, 1022, 1022, 572: 1022, 574: 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 1022, 636: 1022, 638: 1022, 732: 1022, 1022, 735: 1022, 1022, 1022, 743: 1022, 755: 1022, 1022, 1022}, // 2770 {2: 1021, 1021, 1021, 1021, 1021, 8: 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 58: 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 459: 1021, 461: 1021, 1021, 1021, 1021, 469: 1021, 1021, 1021, 1021, 1021, 1021, 478: 1021, 481: 1021, 484: 1021, 486: 1021, 1021, 492: 1021, 1021, 500: 1021, 509: 1021, 529: 1021, 552: 1021, 554: 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 564: 1021, 1021, 1021, 1021, 1021, 1021, 572: 1021, 574: 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 1021, 636: 1021, 638: 1021, 732: 1021, 1021, 735: 1021, 1021, 1021, 743: 1021, 755: 1021, 1021, 1021}, {2: 1019, 1019, 1019, 1019, 1019, 8: 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 58: 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 1019, 459: 1019, 474: 1019, 481: 1019, 487: 1019, 565: 1019}, {2: 1803, 1803, 1803, 1803, 1803, 8: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 58: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 459: 1803, 474: 4071, 565: 1803, 838: 5221}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5224, 834: 5222, 868: 5223}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5224, 834: 5222, 869: 5223}, {896, 896, 7: 896, 57: 896, 458: 896, 460: 896, 466: 896, 468: 896, 476: 896, 896, 479: 896, 896, 896, 483: 896, 488: 896, 896, 896, 498: 896, 896, 501: 896, 896}, // 2775 {7: 5276, 498: 5346}, @@ -9340,25 +9340,25 @@ var ( {891, 891, 7: 891, 57: 891, 458: 891, 460: 891, 466: 891, 468: 891, 891, 891, 476: 891, 891, 479: 891, 891, 891, 483: 891, 488: 891, 891, 891, 498: 891, 891, 891, 891, 891, 891, 891, 891, 891, 508: 891}, // 2780 {887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 460: 887, 465: 887, 887, 468: 887, 887, 887, 474: 887, 5280, 887, 887, 479: 887, 887, 887, 483: 887, 485: 887, 488: 887, 887, 887, 498: 887, 887, 887, 887, 887, 887, 887, 887, 887, 508: 887, 511: 887, 887, 660: 887, 842: 5279}, - {885, 885, 2900, 2748, 2784, 2902, 2675, 885, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 885, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 885, 460: 885, 465: 5237, 885, 468: 885, 885, 885, 476: 885, 885, 479: 885, 885, 885, 483: 885, 488: 885, 885, 885, 498: 885, 885, 885, 885, 885, 885, 885, 885, 885, 508: 885, 648: 5236, 2664, 2665, 2663, 899: 5235, 5234}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 2496, 484: 2495, 553: 2494, 565: 5225, 632: 2490, 648: 3805, 2664, 2665, 2663, 696: 5233, 724: 5228, 738: 3765, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 3767, 3766, 3764, 784: 5227, 787: 5226, 5232, 834: 5222, 868: 5231}, + {885, 885, 2900, 2748, 2784, 2902, 2675, 885, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 885, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 885, 460: 885, 465: 5237, 885, 468: 885, 885, 885, 476: 885, 885, 479: 885, 885, 885, 483: 885, 488: 885, 885, 885, 498: 885, 885, 885, 885, 885, 885, 885, 885, 885, 508: 885, 648: 5236, 2664, 2665, 2663, 900: 5235, 5234}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 2496, 484: 2495, 553: 2494, 565: 5225, 632: 2490, 648: 3805, 2664, 2665, 2663, 696: 5233, 724: 5228, 738: 3765, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 3767, 3766, 3764, 783: 5227, 786: 5226, 5232, 834: 5222, 869: 5231}, {7: 5276, 57: 5277}, {894, 894, 7: 894, 57: 894, 458: 894, 460: 894, 466: 894, 468: 894, 5243, 5244, 476: 894, 894, 479: 894, 894, 894, 483: 894, 488: 894, 894, 894, 498: 894, 894, 5242, 894, 894, 5245, 5241, 5246, 5247, 803: 5240, 809: 5239}, // 2785 - {2: 2900, 2748, 2784, 2902, 2675, 885, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 885, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 465: 5237, 780, 469: 885, 885, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 500: 885, 503: 885, 885, 885, 885, 648: 5236, 2664, 2665, 2663, 753: 3775, 3776, 899: 5235, 5234}, + {2: 2900, 2748, 2784, 2902, 2675, 885, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 885, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 465: 5237, 780, 469: 885, 885, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 500: 885, 503: 885, 885, 885, 885, 648: 5236, 2664, 2665, 2663, 753: 3775, 3776, 900: 5235, 5234}, {889, 889, 7: 889, 57: 889, 458: 889, 460: 889, 466: 889, 468: 889, 889, 889, 476: 889, 889, 479: 889, 889, 889, 483: 889, 488: 889, 889, 889, 498: 889, 889, 889, 889, 889, 889, 889, 889, 889, 508: 889}, {884, 884, 7: 884, 57: 884, 458: 884, 460: 884, 466: 884, 468: 884, 884, 884, 474: 884, 476: 884, 884, 479: 884, 884, 884, 483: 884, 485: 884, 488: 884, 884, 884, 498: 884, 884, 884, 884, 884, 884, 884, 884, 884, 508: 884, 511: 884, 884, 660: 884}, {883, 883, 7: 883, 57: 883, 458: 883, 460: 883, 466: 883, 468: 883, 883, 883, 474: 883, 476: 883, 883, 479: 883, 883, 883, 483: 883, 485: 883, 488: 883, 883, 883, 498: 883, 883, 883, 883, 883, 883, 883, 883, 883, 508: 883, 511: 883, 883, 660: 883}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5238, 2664, 2665, 2663}, // 2790 {882, 882, 7: 882, 57: 882, 458: 882, 460: 882, 466: 882, 468: 882, 882, 882, 474: 882, 476: 882, 882, 479: 882, 882, 882, 483: 882, 485: 882, 488: 882, 882, 882, 498: 882, 882, 882, 882, 882, 882, 882, 882, 882, 508: 882, 511: 882, 882, 660: 882}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5269}, - {503: 853, 890: 5256, 1073: 5260}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5269}, + {503: 853, 891: 5256, 1074: 5260}, {469: 5243, 5244, 503: 5253, 803: 5254}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5250}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5250}, // 2795 - {503: 855, 890: 855}, - {503: 854, 890: 854}, + {503: 855, 891: 855}, + {503: 854, 891: 854}, {2: 851, 851, 851, 851, 851, 8: 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 58: 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 851, 459: 851}, {503: 5249}, {503: 5248}, @@ -9369,21 +9369,21 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5252}, {856, 856, 7: 856, 57: 856, 458: 856, 460: 856, 466: 856, 468: 856, 856, 856, 476: 856, 856, 479: 856, 856, 856, 483: 856, 488: 856, 856, 856, 3234, 494: 3232, 3233, 3231, 3229, 856, 856, 856, 856, 856, 856, 856, 856, 856, 508: 856, 720: 3230, 3228}, // 2805 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5259}, - {503: 853, 890: 5256, 1073: 5255}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5259}, + {503: 853, 891: 5256, 1074: 5255}, {503: 5257}, {503: 852}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5258}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5258}, // 2810 {858, 858, 7: 858, 57: 858, 458: 858, 460: 858, 466: 858, 468: 858, 858, 858, 476: 858, 858, 479: 858, 858, 858, 483: 858, 488: 858, 858, 858, 498: 858, 858, 858, 858, 858, 858, 858, 858, 858, 508: 858, 803: 5240, 809: 5239}, {859, 859, 7: 859, 57: 859, 458: 859, 460: 859, 466: 859, 468: 859, 859, 859, 476: 859, 859, 479: 859, 859, 859, 483: 859, 488: 859, 859, 859, 498: 859, 859, 859, 859, 859, 859, 859, 859, 859, 508: 859, 803: 5240, 809: 5239}, {503: 5261}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5262}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5262}, {458: 5263, 468: 5264, 5243, 5244, 500: 5242, 503: 5245, 5241, 5246, 5247, 803: 5240, 809: 5239}, // 2815 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5268}, {459: 5265}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 5266}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 5266}, {7: 4027, 57: 5267}, {860, 860, 7: 860, 57: 860, 458: 860, 460: 860, 466: 860, 468: 860, 860, 860, 476: 860, 860, 479: 860, 860, 860, 483: 860, 488: 860, 860, 860, 498: 860, 860, 860, 860, 860, 860, 860, 860, 860, 508: 860}, // 2820 @@ -9391,25 +9391,25 @@ var ( {864, 864, 7: 864, 57: 864, 458: 5270, 460: 864, 466: 864, 468: 5271, 5243, 5244, 476: 864, 864, 479: 864, 864, 864, 483: 864, 488: 864, 864, 864, 498: 864, 864, 5242, 864, 864, 5245, 5241, 5246, 5247, 508: 864, 803: 5240, 809: 5239}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5275}, {459: 5272}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 5273}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 5273}, // 2825 {7: 4027, 57: 5274}, {862, 862, 7: 862, 57: 862, 458: 862, 460: 862, 466: 862, 468: 862, 862, 862, 476: 862, 862, 479: 862, 862, 862, 483: 862, 488: 862, 862, 862, 498: 862, 862, 862, 862, 862, 862, 862, 862, 862, 508: 862}, {863, 863, 7: 863, 57: 863, 458: 863, 460: 863, 466: 863, 468: 863, 863, 863, 476: 863, 863, 479: 863, 863, 863, 483: 863, 488: 863, 863, 863, 3234, 494: 3232, 3233, 3231, 3229, 863, 863, 863, 863, 863, 863, 863, 863, 863, 508: 863, 720: 3230, 3228}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5232, 834: 5278}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5232, 834: 5278}, {888, 888, 7: 888, 57: 888, 458: 888, 460: 888, 466: 888, 468: 888, 888, 888, 476: 888, 888, 479: 888, 888, 888, 483: 888, 488: 888, 888, 888, 498: 888, 888, 888, 888, 888, 888, 888, 888, 888, 508: 888}, // 2830 {895, 895, 7: 895, 57: 895, 458: 895, 460: 895, 466: 895, 468: 895, 476: 895, 895, 479: 895, 895, 895, 483: 895, 488: 895, 895, 895, 498: 895, 895, 501: 895, 895}, - {885, 885, 2900, 2748, 2784, 2902, 2675, 885, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 885, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 885, 460: 885, 465: 5237, 885, 468: 885, 885, 885, 474: 885, 476: 885, 885, 479: 885, 885, 885, 483: 885, 485: 885, 488: 885, 885, 885, 498: 885, 885, 885, 885, 885, 885, 885, 885, 885, 508: 885, 511: 885, 885, 648: 5236, 2664, 2665, 2663, 660: 885, 899: 5235, 5284}, + {885, 885, 2900, 2748, 2784, 2902, 2675, 885, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 885, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 885, 460: 885, 465: 5237, 885, 468: 885, 885, 885, 474: 885, 476: 885, 885, 479: 885, 885, 885, 483: 885, 485: 885, 488: 885, 885, 885, 498: 885, 885, 885, 885, 885, 885, 885, 885, 885, 508: 885, 511: 885, 885, 648: 5236, 2664, 2665, 2663, 660: 885, 900: 5235, 5284}, {459: 5281}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 775: 5282}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 774: 5282}, {7: 4626, 57: 5283}, // 2835 {886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 886, 465: 886, 886, 468: 886, 886, 886, 474: 886, 476: 886, 886, 479: 886, 886, 886, 483: 886, 886, 886, 488: 886, 886, 886, 498: 886, 886, 886, 886, 886, 886, 886, 886, 886, 508: 886, 511: 886, 886, 541: 886, 553: 886, 632: 886, 635: 886, 646: 886, 660: 886}, - {1810, 1810, 7: 1810, 57: 1810, 458: 1810, 460: 1810, 466: 1810, 468: 1810, 1810, 1810, 474: 1810, 476: 1810, 1810, 479: 1810, 1810, 1810, 483: 1810, 485: 1810, 488: 1810, 1810, 1810, 498: 1810, 1810, 1810, 1810, 1810, 1810, 1810, 1810, 1810, 508: 1810, 511: 1810, 1810, 660: 5286, 907: 5285, 1150: 5287}, + {1810, 1810, 7: 1810, 57: 1810, 458: 1810, 460: 1810, 466: 1810, 468: 1810, 1810, 1810, 474: 1810, 476: 1810, 1810, 479: 1810, 1810, 1810, 483: 1810, 485: 1810, 488: 1810, 1810, 1810, 498: 1810, 1810, 1810, 1810, 1810, 1810, 1810, 1810, 1810, 508: 1810, 511: 1810, 1810, 660: 5286, 908: 5285, 1151: 5287}, {1809, 1809, 7: 1809, 57: 1809, 458: 1809, 460: 1809, 466: 1809, 468: 1809, 1809, 1809, 474: 1809, 476: 1809, 1809, 479: 1809, 1809, 1809, 483: 1809, 485: 1809, 488: 1809, 1809, 1809, 498: 1809, 1809, 1809, 1809, 1809, 1809, 1809, 1809, 1809, 508: 1809, 511: 1809, 1809}, {212: 5328}, - {866, 866, 7: 866, 57: 866, 458: 866, 460: 866, 466: 866, 468: 866, 866, 866, 474: 5290, 476: 866, 866, 479: 866, 866, 866, 483: 866, 485: 5291, 488: 866, 866, 866, 498: 866, 866, 866, 866, 866, 866, 866, 866, 866, 508: 866, 511: 5289, 866, 923: 5293, 5292, 1039: 5294, 5288}, + {866, 866, 7: 866, 57: 866, 458: 866, 460: 866, 466: 866, 468: 866, 866, 866, 474: 5290, 476: 866, 866, 479: 866, 866, 866, 483: 866, 485: 5291, 488: 866, 866, 866, 498: 866, 866, 866, 866, 866, 866, 866, 866, 866, 508: 866, 511: 5289, 866, 924: 5293, 5292, 1040: 5294, 5288}, // 2840 {981, 981, 7: 981, 57: 981, 458: 981, 460: 981, 466: 981, 468: 981, 981, 981, 476: 981, 981, 479: 981, 981, 981, 483: 981, 488: 981, 981, 981, 498: 981, 981, 981, 981, 981, 981, 981, 981, 981, 508: 981, 512: 5309, 1294: 5310}, {563: 4299, 635: 4300, 813: 5308}, @@ -9418,7 +9418,7 @@ var ( {459: 878, 479: 5296, 1205: 5297}, // 2845 {868, 868, 7: 868, 57: 868, 458: 868, 460: 868, 466: 868, 468: 868, 868, 868, 474: 868, 476: 868, 868, 479: 868, 868, 868, 483: 868, 485: 868, 488: 868, 868, 868, 498: 868, 868, 868, 868, 868, 868, 868, 868, 868, 508: 868, 511: 868, 868}, - {865, 865, 7: 865, 57: 865, 458: 865, 460: 865, 466: 865, 468: 865, 865, 865, 474: 5290, 476: 865, 865, 479: 865, 865, 865, 483: 865, 485: 5291, 488: 865, 865, 865, 498: 865, 865, 865, 865, 865, 865, 865, 865, 865, 508: 865, 511: 5289, 865, 923: 5295, 5292}, + {865, 865, 7: 865, 57: 865, 458: 865, 460: 865, 466: 865, 468: 865, 865, 865, 474: 5290, 476: 865, 865, 479: 865, 865, 865, 483: 865, 485: 5291, 488: 865, 865, 865, 498: 865, 865, 865, 865, 865, 865, 865, 865, 865, 508: 865, 511: 5289, 865, 924: 5295, 5292}, {867, 867, 7: 867, 57: 867, 458: 867, 460: 867, 466: 867, 468: 867, 867, 867, 474: 867, 476: 867, 867, 479: 867, 867, 867, 483: 867, 485: 867, 488: 867, 867, 867, 498: 867, 867, 867, 867, 867, 867, 867, 867, 867, 508: 867, 511: 867, 867}, {490: 5302, 499: 5303, 503: 5301}, {459: 5298}, @@ -9445,7 +9445,7 @@ var ( {459: 975}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 5317, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5316}, {57: 974, 355: 5325, 491: 3234, 494: 3232, 3233, 3231, 3229, 510: 5324, 720: 3230, 3228, 1295: 5323}, - {971, 971, 7: 971, 57: 971, 207: 5319, 458: 971, 460: 971, 466: 971, 468: 971, 971, 971, 476: 971, 971, 479: 971, 971, 971, 483: 971, 488: 971, 971, 971, 498: 971, 971, 971, 971, 971, 971, 971, 971, 971, 508: 971, 1091: 5318}, + {971, 971, 7: 971, 57: 971, 207: 5319, 458: 971, 460: 971, 466: 971, 468: 971, 971, 971, 476: 971, 971, 479: 971, 971, 971, 483: 971, 488: 971, 971, 971, 498: 971, 971, 971, 971, 971, 971, 971, 971, 971, 508: 971, 1092: 5318}, // 2870 {979, 979, 7: 979, 57: 979, 458: 979, 460: 979, 466: 979, 468: 979, 979, 979, 476: 979, 979, 479: 979, 979, 979, 483: 979, 488: 979, 979, 979, 498: 979, 979, 979, 979, 979, 979, 979, 979, 979, 508: 979}, {459: 5320}, @@ -9456,38 +9456,38 @@ var ( {57: 5326}, {57: 973}, {57: 972}, - {971, 971, 7: 971, 57: 971, 207: 5319, 458: 971, 460: 971, 466: 971, 468: 971, 971, 971, 476: 971, 971, 479: 971, 971, 971, 483: 971, 488: 971, 971, 971, 498: 971, 971, 971, 971, 971, 971, 971, 971, 971, 508: 971, 1091: 5327}, + {971, 971, 7: 971, 57: 971, 207: 5319, 458: 971, 460: 971, 466: 971, 468: 971, 971, 971, 476: 971, 971, 479: 971, 971, 971, 483: 971, 488: 971, 971, 971, 498: 971, 971, 971, 971, 971, 971, 971, 971, 971, 508: 971, 1092: 5327}, {980, 980, 7: 980, 57: 980, 458: 980, 460: 980, 466: 980, 468: 980, 980, 980, 476: 980, 980, 479: 980, 980, 980, 483: 980, 488: 980, 980, 980, 498: 980, 980, 980, 980, 980, 980, 980, 980, 980, 508: 980}, // 2880 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5329}, {1808, 1808, 7: 1808, 57: 1808, 458: 1808, 460: 1808, 466: 1808, 468: 1808, 1808, 1808, 474: 1808, 476: 1808, 1808, 479: 1808, 1808, 1808, 483: 1808, 485: 1808, 488: 1808, 1808, 1808, 3234, 494: 3232, 3233, 3231, 3229, 1808, 1808, 1808, 1808, 1808, 1808, 1808, 1808, 1808, 508: 1808, 511: 1808, 1808, 720: 3230, 3228}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5331}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5331}, {469: 5243, 5244, 500: 5242, 503: 5245, 5241, 5246, 5247, 508: 5332, 803: 5240, 809: 5239}, {893, 893, 7: 893, 57: 893, 458: 893, 460: 893, 466: 893, 468: 893, 476: 893, 893, 479: 893, 893, 893, 483: 893, 488: 893, 893, 893, 498: 893, 893, 501: 893, 893}, // 2885 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 870: 5335, 908: 5336}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 871: 5335, 909: 5336}, {482: 5344}, {2273, 2273, 7: 2273, 468: 2273, 480: 2273, 489: 2273, 2273}, - {237, 237, 7: 5337, 468: 237, 480: 237, 489: 2624, 237, 778: 2625, 5338}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 870: 5343}, + {237, 237, 7: 5337, 468: 237, 480: 237, 489: 2624, 237, 777: 2625, 5338}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 871: 5343}, // 2890 {1258, 1258, 468: 1258, 480: 1258, 490: 2627, 753: 2628, 797: 5339}, - {848, 848, 468: 848, 480: 5340, 1048: 5341}, - {493: 2638, 564: 2640, 722: 2637, 730: 2639, 860: 5342}, + {848, 848, 468: 848, 480: 5340, 1049: 5341}, + {493: 2638, 564: 2640, 722: 2637, 730: 2639, 861: 5342}, {241, 241, 468: 241}, {847, 847, 468: 847}, // 2895 {2272, 2272, 7: 2272, 468: 2272, 480: 2272, 489: 2272, 2272}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5345}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5345}, {2274, 2274, 7: 2274, 468: 2274, 480: 2274, 489: 2274, 2274}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 870: 5335, 908: 5347}, - {237, 237, 7: 5337, 468: 237, 489: 2624, 778: 2625, 5348}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 871: 5335, 909: 5347}, + {237, 237, 7: 5337, 468: 237, 489: 2624, 777: 2625, 5348}, // 2900 {240, 240, 468: 240}, {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, 379, 379, 379, 379, 379, 379, 379, 58: 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 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: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5351}, {378, 378}, - {31: 5364, 110: 5354, 125: 5357, 142: 556, 180: 5356, 186: 5367, 195: 5365, 211: 5358, 222: 5362, 242: 5366, 245: 5359, 529: 5363, 553: 5353, 1123: 5361, 1192: 5355, 1223: 5360}, + {31: 5364, 110: 5354, 125: 5357, 142: 556, 180: 5356, 186: 5367, 195: 5365, 211: 5358, 222: 5362, 242: 5366, 245: 5359, 529: 5363, 553: 5353, 1124: 5361, 1193: 5355, 1223: 5360}, // 2905 {1989, 1989, 1989, 1989, 1989, 1989, 1989, 8: 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 58: 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 1989, 460: 1989, 552: 1989}, {1988, 1988, 1988, 1988, 1988, 1988, 1988, 8: 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 58: 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 460: 1988, 552: 1988}, @@ -9498,7 +9498,7 @@ var ( {203: 5374}, {560, 560}, {142: 5373}, - {547, 547, 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 460: 547, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 4192, 1122: 5368}, + {547, 547, 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 460: 547, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 4192, 1123: 5368}, {557, 557}, // 2915 {142: 555}, @@ -9514,7 +9514,7 @@ var ( {542, 542}, // 2925 {559, 559}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5375, 2664, 2665, 2663, 935: 5376}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5375, 2664, 2665, 2663, 936: 5376}, {565, 565, 7: 565}, {561, 561, 7: 5377}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5378, 2664, 2665, 2663}, @@ -9527,7 +9527,7 @@ var ( // 2935 {553: 5452}, {312: 5446}, - {138: 5443, 209: 5445, 321: 5441, 350: 5442, 898: 5444}, + {138: 5443, 209: 5445, 321: 5441, 350: 5442, 899: 5444}, {191: 5438, 194: 5437}, {553: 5396}, // 2940 @@ -9541,9 +9541,9 @@ var ( {675, 675}, {676, 676}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5397}, - {661: 5398, 914: 5399}, + {661: 5398, 915: 5399}, // 2950 - {156: 5401, 160: 5402, 553: 2004, 930: 5400}, + {156: 5401, 160: 5402, 553: 2004, 931: 5400}, {677, 677}, {553: 5404}, {110: 2003, 553: 2003}, @@ -9553,11 +9553,11 @@ var ( {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 552: 4701, 769: 5405}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5406}, {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, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 458: 425, 5410, 425, 464: 425, 425, 425, 425, 474: 425, 425, 484: 425, 425, 425, 492: 425, 507: 5409, 553: 425, 632: 425, 634: 425, 425, 1217: 5408, 1290: 5407}, - {385, 385, 4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 458: 385, 385, 385, 464: 4130, 385, 4167, 2106, 474: 385, 385, 484: 385, 4844, 2106, 492: 385, 553: 385, 632: 385, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 780: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 903: 5425, 997: 5424}, + {385, 385, 4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 458: 385, 385, 385, 464: 4130, 385, 4167, 2106, 474: 385, 385, 484: 385, 4844, 2106, 492: 385, 553: 385, 632: 385, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 904: 5425, 998: 5424}, // 2960 - {2109, 2109, 458: 5418, 1063: 5417}, + {2109, 2109, 458: 5418, 1064: 5417}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5416}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 507: 5411, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 648: 4024, 2664, 2665, 2663, 662: 2323, 2323, 729: 4345, 819: 4687, 831: 4819, 884: 4820, 948: 4821, 1120: 5412}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 507: 5411, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 4478, 635: 2323, 648: 4024, 2664, 2665, 2663, 662: 2323, 2323, 729: 4345, 819: 4687, 831: 4819, 885: 4820, 949: 4821, 1121: 5412}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5414}, {7: 4823, 57: 5413}, // 2965 @@ -9574,24 +9574,24 @@ var ( {2107, 2107}, // 2975 {2108, 2108}, - {2104, 2104, 458: 2104, 2104, 2104, 465: 2104, 474: 2104, 5427, 484: 2104, 492: 2104, 553: 2104, 632: 2104, 1076: 5426}, - {384, 384, 4: 4131, 4133, 389, 4846, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 458: 384, 384, 384, 464: 4130, 384, 4167, 2106, 474: 384, 384, 484: 384, 4844, 2106, 492: 384, 553: 384, 632: 384, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 780: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4845}, - {2052, 2052, 458: 2052, 2052, 2052, 465: 2052, 474: 5021, 484: 2052, 492: 5022, 553: 2052, 632: 2052, 1017: 5428}, + {2104, 2104, 458: 2104, 2104, 2104, 465: 2104, 474: 2104, 5427, 484: 2104, 492: 2104, 553: 2104, 632: 2104, 1077: 5426}, + {384, 384, 4: 4131, 4133, 389, 4846, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 458: 384, 384, 384, 464: 4130, 384, 4167, 2106, 474: 384, 384, 484: 384, 4844, 2106, 492: 384, 553: 384, 632: 384, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4845}, + {2052, 2052, 458: 2052, 2052, 2052, 465: 2052, 474: 5021, 484: 2052, 492: 5022, 553: 2052, 632: 2052, 1018: 5428}, {646: 4916}, // 2980 - {2049, 2049, 458: 2049, 2049, 2049, 465: 5430, 484: 2049, 553: 2049, 632: 2049, 1151: 5429}, - {2047, 2047, 458: 2047, 2497, 2496, 484: 2495, 553: 2494, 632: 2490, 696: 5435, 738: 5433, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5434, 5432, 3764, 1172: 5431}, + {2049, 2049, 458: 2049, 2049, 2049, 465: 5430, 484: 2049, 553: 2049, 632: 2049, 1152: 5429}, + {2047, 2047, 458: 2047, 2497, 2496, 484: 2495, 553: 2494, 632: 2490, 696: 5435, 738: 5433, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5434, 5432, 3764, 1173: 5431}, {2048, 2048, 458: 2048, 2048, 2048, 484: 2048, 553: 2048, 632: 2048}, - {2109, 2109, 458: 5418, 1063: 5436}, + {2109, 2109, 458: 5418, 1064: 5436}, {2046, 2046, 458: 2046}, // 2985 {2045, 2045, 458: 2045, 466: 781, 476: 781, 781}, {2044, 2044, 458: 2044}, {2043, 2043, 458: 2043, 466: 780, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, {2111, 2111}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5375, 2664, 2665, 2663, 935: 5440}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5375, 2664, 2665, 2663, 936: 5440}, // 2990 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5375, 2664, 2665, 2663, 935: 5439}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5375, 2664, 2665, 2663, 936: 5439}, {679, 679, 7: 5377}, {680, 680, 7: 5377}, {682, 682}, @@ -9601,18 +9601,18 @@ var ( {672, 672}, {671, 671}, {250: 5447}, - {493: 2638, 722: 3934, 748: 5449, 1059: 5448}, + {493: 2638, 722: 3934, 748: 5449, 1060: 5448}, // 3000 {685, 685, 7: 5450}, {661, 661, 7: 661}, {493: 2638, 722: 3934, 748: 5451}, {660, 660, 7: 660}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 5453}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 5453}, // 3005 {686, 686, 7: 3808}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5458}, {483: 5456}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 5457}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 5457}, {678, 678, 7: 3808}, // 3010 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5459, 2664, 2665, 2663}, @@ -9621,10 +9621,10 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5462, 2664, 2665, 2663}, {689, 689}, // 3015 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 5476}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 5476}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5465}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5466, 2664, 2665, 2663}, - {690, 690, 459: 5469, 1035: 5468, 1197: 5467}, + {690, 690, 459: 5469, 1036: 5468, 1198: 5467}, {687, 687, 7: 5474}, // 3020 {664, 664, 7: 664}, @@ -9634,22 +9634,22 @@ var ( {57: 5473}, // 3025 {662, 662, 7: 662}, - {459: 5469, 1035: 5475}, + {459: 5469, 1036: 5475}, {663, 663, 7: 663}, {691, 691, 7: 3808}, - {153: 1591, 361: 5490, 384: 5491, 639: 1591, 1142: 5489}, + {153: 1591, 361: 5490, 384: 5491, 639: 1591, 1143: 5489}, // 3030 {695, 695, 153: 1433, 249: 5483, 5482, 639: 1433}, {670, 670, 153: 1415, 639: 1415}, {153: 5481}, {692, 692}, - {237, 237, 489: 2624, 493: 2638, 722: 3934, 748: 5487, 778: 2625, 5486}, + {237, 237, 489: 2624, 493: 2638, 722: 3934, 748: 5487, 777: 2625, 5486}, // 3035 {360: 5484}, - {493: 2638, 722: 3934, 748: 5449, 1059: 5485}, + {493: 2638, 722: 3934, 748: 5449, 1060: 5485}, {684, 684, 7: 5450}, {694, 694}, - {237, 237, 489: 2624, 778: 2625, 5488}, + {237, 237, 489: 2624, 777: 2625, 5488}, // 3040 {693, 693}, {683, 683}, @@ -9667,12 +9667,12 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5602, 482: 1750, 639: 1750, 647: 1750, 5601, 2664, 2665, 2663}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 5599, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 482: 1713, 639: 1713, 647: 1713, 5509, 2664, 2665, 2663, 817: 5552}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 482: 1707, 639: 1707, 647: 1707, 5509, 2664, 2665, 2663, 817: 5596}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 464: 5592, 482: 1705, 529: 3645, 639: 1705, 647: 1705, 3377, 2664, 2665, 2663, 725: 3644, 794: 5591}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 464: 5592, 482: 1705, 529: 3645, 639: 1705, 647: 1705, 3377, 2664, 2665, 2663, 725: 3644, 793: 5591}, // 3055 {479: 5581, 482: 5580, 639: 1700, 647: 1700}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 5533, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 464: 5577, 482: 1691, 638: 5575, 1691, 647: 1691, 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 5540, 5538, 844: 5536, 1101: 5576, 1266: 5574}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 5533, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 464: 5577, 482: 1691, 638: 5575, 1691, 647: 1691, 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 5540, 5538, 844: 5536, 1102: 5576, 1266: 5574}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 5572, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 482: 1689, 639: 1689, 647: 1689, 5509, 2664, 2665, 2663, 817: 5549}, - {173: 5557, 482: 1672, 639: 1672, 647: 1672, 658: 5558, 904: 5556, 951: 5555}, + {173: 5557, 482: 1672, 639: 1672, 647: 1672, 658: 5558, 905: 5556, 952: 5555}, {773, 773, 7: 5545}, // 3060 {164: 5531}, @@ -9681,7 +9681,7 @@ var ( {482: 5518, 647: 5519, 810: 5522}, {482: 5518, 647: 5519, 810: 5520}, // 3065 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 464: 5517, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 5516, 1159: 5515}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 464: 5517, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 5516, 1160: 5515}, {720, 720, 7: 720}, {727, 727, 7: 727}, {726, 726, 7: 726}, @@ -9691,18 +9691,18 @@ var ( {2: 743, 743, 743, 743, 743, 8: 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 58: 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 461: 743, 743, 743, 743, 469: 743, 743, 743, 743, 743, 478: 743, 484: 743, 486: 743, 492: 743, 743, 529: 743, 552: 743, 554: 743, 743, 743, 743, 743, 743, 743, 743, 743, 564: 743, 743, 743, 743, 743, 743, 572: 743, 574: 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 743, 636: 743}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5521}, {732, 732, 7: 732, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5525, 845: 5526}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5525, 845: 5526}, // 3075 {746, 746, 2900, 2748, 2784, 2902, 2675, 746, 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 464: 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 3237, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3750, 3140, 3223, 3139, 3136}, {747, 747, 7: 747}, {745, 745, 7: 745}, {733, 733, 7: 733}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5525, 845: 5528}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5525, 845: 5528}, // 3080 {737, 737, 7: 737}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5530, 2664, 2665, 2663}, {482: 741, 647: 741}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 5533, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 638: 5535, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 5540, 5538, 844: 5536, 1101: 5534}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 5533, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 638: 5535, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 5540, 5538, 844: 5536, 1102: 5534}, {704, 704, 7: 704, 554: 1779, 637: 704, 654: 1779}, // 3085 {763, 763, 554: 1613, 637: 763, 654: 1613}, @@ -9714,22 +9714,22 @@ var ( {699, 699, 7: 699, 637: 699}, {698, 698, 7: 698, 637: 698}, {697, 697, 7: 697, 637: 697}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 5542, 5538}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 5542, 5538}, {696, 696, 7: 696, 637: 696}, // 3095 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5113, 848: 5544}, {764, 764, 7: 5115}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 5498, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 5501, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 5546, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 5547, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 5502, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 486: 3721, 554: 5512, 574: 5511, 634: 3719, 648: 5509, 2664, 2665, 2663, 759: 5513, 817: 5510, 958: 5548}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 5498, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 5501, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 5546, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 5547, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 5502, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 486: 3721, 554: 5512, 574: 5511, 634: 3719, 648: 5509, 2664, 2665, 2663, 759: 5513, 817: 5510, 959: 5548}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 482: 1713, 639: 1713, 647: 1713, 5509, 2664, 2665, 2663, 817: 5552}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 482: 1689, 639: 1689, 647: 1689, 5509, 2664, 2665, 2663, 817: 5549}, // 3100 {719, 719, 7: 719}, {482: 5518, 647: 5519, 810: 5550}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5525, 845: 5551}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5525, 845: 5551}, {735, 735, 7: 735}, {482: 5518, 647: 5519, 810: 5553}, // 3105 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5525, 845: 5554}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5525, 845: 5554}, {736, 736, 7: 736}, {768, 768, 7: 5570}, {757, 757, 7: 757}, @@ -9737,7 +9737,7 @@ var ( // 3110 {149: 5560, 674: 5559}, {754, 754, 7: 754}, - {753, 753, 7: 753, 660: 5286, 907: 5561}, + {753, 753, 7: 753, 660: 5286, 908: 5561}, {752, 752, 7: 752}, {207: 5564, 368: 5566, 658: 5565, 1212: 5563}, // 3115 @@ -9749,9 +9749,9 @@ var ( // 3120 {749, 749, 7: 749}, {751, 751, 7: 751}, - {173: 5557, 658: 5558, 904: 5571}, + {173: 5557, 658: 5558, 905: 5571}, {756, 756, 7: 756}, - {173: 5557, 482: 1672, 639: 1672, 647: 1672, 658: 5558, 904: 5556, 951: 5573}, + {173: 5557, 482: 1672, 639: 1672, 647: 1672, 658: 5558, 905: 5556, 952: 5573}, // 3125 {769, 769, 7: 5570}, {765, 765}, @@ -9759,17 +9759,17 @@ var ( {759, 759}, {758, 758}, // 3130 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 5540, 5538, 844: 5579}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 5540, 5538, 844: 5579}, {760, 760, 7: 5541}, - {14: 5586, 461: 5585, 1077: 5590}, + {14: 5586, 461: 5585, 1078: 5590}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5582}, {482: 5583}, // 3135 - {14: 5586, 461: 5585, 1077: 5584}, + {14: 5586, 461: 5585, 1078: 5584}, {771, 771}, {708, 708}, {459: 5587}, - {461: 5145, 871: 5588}, + {461: 5145, 872: 5588}, // 3140 {57: 5589}, {707, 707}, @@ -9784,22 +9784,22 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5598}, // 3150 {734, 734, 7: 734, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {173: 5557, 482: 1672, 639: 1672, 647: 1672, 658: 5558, 904: 5556, 951: 5600}, + {173: 5557, 482: 1672, 639: 1672, 647: 1672, 658: 5558, 905: 5556, 952: 5600}, {770, 770, 7: 5570}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 883: 5611}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 883: 5603}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 884: 5611}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 884: 5603}, // 3155 {482: 5518, 647: 5519, 810: 5609}, {471: 5606, 482: 740, 639: 5605, 647: 740}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 883: 5608}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 883: 5607}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 884: 5608}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5604, 2664, 2665, 2663, 884: 5607}, {482: 738, 647: 738}, // 3160 {482: 739, 647: 739}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5525, 845: 5610}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5525, 845: 5610}, {766, 766}, {482: 5518, 647: 5519, 810: 5612}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5525, 845: 5613}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 5524, 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 5523, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5525, 845: 5613}, // 3165 {767, 767}, {637: 5623}, @@ -9822,10 +9822,10 @@ var ( {461: 5629}, {775, 775}, {459: 2497, 484: 2495, 553: 2494, 632: 2490, 696: 5641, 738: 5640, 2491, 2492, 2493, 5642}, - {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3432, 732: 3430, 3431, 768: 5634, 772: 5635, 916: 5637, 946: 5639}, - {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3432, 732: 3430, 3431, 768: 5634, 772: 5635, 916: 5637, 946: 5638}, + {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3432, 732: 3430, 3431, 768: 5634, 771: 5635, 917: 5637, 947: 5639}, + {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3432, 732: 3430, 3431, 768: 5634, 771: 5635, 917: 5637, 947: 5638}, // 3185 - {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3432, 732: 3430, 3431, 768: 5634, 772: 5635, 916: 5637, 946: 5636}, + {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3432, 732: 3430, 3431, 768: 5634, 771: 5635, 917: 5637, 947: 5636}, {2: 1204, 1204, 1204, 1204, 1204, 8: 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 58: 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 459: 1204, 461: 1204, 1204, 1204, 1204, 469: 1204, 1204, 1204, 1204, 1204, 478: 1204, 484: 1204, 486: 1204, 492: 1204, 1204, 500: 1204, 509: 1204, 529: 1204, 552: 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 564: 1204, 1204, 1204, 1204, 1204, 1204, 572: 1204, 574: 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 1204, 632: 1204, 636: 1204, 638: 1204, 732: 1204, 1204, 735: 1204, 1204, 1204, 743: 1204, 755: 1204, 1204, 1204}, {459: 1200, 484: 1200, 553: 1200, 632: 1200}, {459: 777, 484: 777, 553: 777, 632: 777}, @@ -9843,77 +9843,77 @@ var ( {466: 780, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, {7: 5661, 459: 962, 484: 962, 553: 962, 632: 962, 641: 962, 731: 962}, // 3200 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5650, 2664, 2665, 2663, 913: 5649, 1140: 5660}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5650, 2664, 2665, 2663, 914: 5649, 1141: 5660}, {7: 959, 459: 959, 484: 959, 553: 959, 632: 959, 641: 959, 731: 959}, {459: 5651, 465: 2248, 1200: 5652}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5656, 2664, 2665, 2663, 1199: 5655}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5656, 2664, 2665, 2663, 856: 5655}, {465: 5653}, // 3205 {459: 2497, 696: 5654}, {7: 958, 459: 958, 484: 958, 553: 958, 632: 958, 641: 958, 731: 958}, {7: 5658, 57: 5657}, - {7: 2246, 57: 2246}, + {2246, 2246, 7: 2246, 57: 2246, 460: 2246}, {465: 2247}, // 3210 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5659, 2664, 2665, 2663}, - {7: 2245, 57: 2245}, + {2245, 2245, 7: 2245, 57: 2245, 460: 2245}, {7: 5661, 459: 961, 484: 961, 553: 961, 632: 961, 641: 961, 731: 961}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5650, 2664, 2665, 2663, 913: 5662}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5650, 2664, 2665, 2663, 914: 5662}, {7: 960, 459: 960, 484: 960, 553: 960, 632: 960, 641: 960, 731: 960}, // 3215 {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 5664}, {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2630, 834, 483: 834, 488: 2631, 754: 2632, 815: 5665}, - {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 866: 5666}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 867: 5667}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 867: 5666}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5667}, {966, 966, 57: 966, 458: 966, 460: 966, 466: 966, 468: 966, 476: 966, 966}, // 3220 {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2630, 834, 483: 834, 488: 2631, 754: 2632, 815: 5669}, - {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 866: 5670}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 867: 5671}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 867: 5670}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5671}, {967, 967, 57: 967, 458: 967, 460: 967, 466: 967, 468: 967, 476: 967, 967}, {646: 5679}, // 3225 {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 5675}, {812, 812, 57: 812, 458: 812, 460: 812, 466: 812, 468: 812, 476: 812, 812, 479: 812, 812, 812, 483: 812, 488: 812, 490: 812, 501: 812, 812}, {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2630, 834, 483: 834, 488: 2631, 754: 2632, 815: 5676}, - {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 866: 5677}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 867: 5678}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 867: 5677}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5678}, // 3230 {968, 968, 57: 968, 458: 968, 460: 968, 466: 968, 468: 968, 476: 968, 968}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 880: 5680}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 2656, 851: 3129, 881: 5680}, {1813, 1813, 7: 3298, 57: 1813, 458: 1813, 460: 1813, 466: 1813, 468: 1813, 476: 1813, 1813, 479: 1813, 1813, 1813, 483: 1813, 488: 1813, 490: 1813, 501: 1813, 1813}, - {237, 237, 57: 237, 458: 237, 460: 237, 466: 237, 468: 237, 476: 237, 237, 479: 237, 237, 237, 483: 237, 488: 237, 2624, 237, 499: 237, 778: 2625, 5706}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5232, 834: 5222, 868: 5691, 1178: 5690, 1292: 5689}, + {237, 237, 57: 237, 458: 237, 460: 237, 466: 237, 468: 237, 476: 237, 237, 479: 237, 237, 237, 483: 237, 488: 237, 2624, 237, 499: 237, 777: 2625, 5706}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5232, 834: 5222, 869: 5691, 1179: 5690, 1292: 5689}, // 3235 - {813, 813, 57: 813, 458: 813, 460: 813, 466: 813, 468: 813, 476: 813, 813, 479: 813, 813, 813, 483: 813, 488: 813, 490: 813, 499: 5672, 922: 5674, 945: 5684}, + {813, 813, 57: 813, 458: 813, 460: 813, 466: 813, 468: 813, 476: 813, 813, 479: 813, 813, 813, 483: 813, 488: 813, 490: 813, 499: 5672, 923: 5674, 946: 5684}, {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 5685}, {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2630, 834, 483: 834, 488: 2631, 754: 2632, 815: 5686}, - {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 866: 5687}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 867: 5688}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3796, 481: 806, 483: 3797, 867: 5687}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5688}, // 3240 {969, 969, 57: 969, 458: 969, 460: 969, 466: 969, 468: 969, 476: 969, 969}, - {237, 237, 57: 237, 458: 237, 460: 237, 466: 237, 468: 237, 476: 237, 237, 479: 237, 237, 237, 483: 237, 488: 237, 2624, 237, 499: 237, 501: 237, 237, 778: 2625, 5692}, + {237, 237, 57: 237, 458: 237, 460: 237, 466: 237, 468: 237, 476: 237, 237, 479: 237, 237, 237, 483: 237, 488: 237, 2624, 237, 499: 237, 501: 237, 237, 777: 2625, 5692}, {957, 957, 57: 957, 458: 957, 460: 957, 466: 957, 468: 957, 476: 957, 957, 479: 957, 957, 957, 483: 957, 488: 957, 957, 957, 499: 957}, {897, 897, 7: 5276, 57: 897, 458: 897, 460: 897, 466: 897, 468: 897, 476: 897, 897, 479: 897, 897, 897, 483: 897, 488: 897, 897, 897, 499: 897, 501: 897, 897}, - {813, 813, 57: 813, 458: 813, 460: 813, 466: 813, 468: 813, 476: 813, 813, 479: 813, 813, 813, 483: 813, 488: 813, 490: 813, 499: 5672, 501: 813, 813, 922: 5674, 945: 5693}, + {813, 813, 57: 813, 458: 813, 460: 813, 466: 813, 468: 813, 476: 813, 813, 479: 813, 813, 813, 483: 813, 488: 813, 490: 813, 499: 5672, 501: 813, 813, 923: 5674, 946: 5693}, // 3245 - {1812, 1812, 57: 1812, 458: 1812, 460: 1812, 466: 1812, 468: 1812, 476: 1812, 1812, 479: 1812, 1812, 1812, 483: 1812, 488: 1812, 490: 1812, 501: 1812, 5694, 1198: 5695}, + {1812, 1812, 57: 1812, 458: 1812, 460: 1812, 466: 1812, 468: 1812, 476: 1812, 1812, 479: 1812, 1812, 1812, 483: 1812, 488: 1812, 490: 1812, 501: 1812, 5694, 1199: 5695}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5705}, {956, 956, 57: 956, 458: 956, 460: 956, 466: 956, 468: 956, 476: 956, 956, 479: 956, 956, 956, 483: 956, 488: 956, 490: 956, 501: 5697, 1315: 5696}, {982, 982, 57: 982, 458: 982, 460: 982, 466: 982, 468: 982, 476: 982, 982, 479: 982, 982, 982, 483: 982, 488: 982, 490: 982}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3286, 2664, 2665, 2663, 906: 5700, 1136: 5699, 1316: 5698}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3286, 2664, 2665, 2663, 907: 5700, 1137: 5699, 1316: 5698}, // 3250 {955, 955, 7: 5703, 57: 955, 458: 955, 460: 955, 466: 955, 468: 955, 476: 955, 955, 479: 955, 955, 955, 483: 955, 488: 955, 490: 955}, {954, 954, 7: 954, 57: 954, 458: 954, 460: 954, 466: 954, 468: 954, 476: 954, 954, 479: 954, 954, 954, 483: 954, 488: 954, 490: 954}, {465: 5701}, - {459: 3287, 1138: 5702}, + {459: 3287, 1139: 5702}, {952, 952, 7: 952, 57: 952, 458: 952, 460: 952, 466: 952, 468: 952, 476: 952, 952, 479: 952, 952, 952, 483: 952, 488: 952, 490: 952}, // 3255 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3286, 2664, 2665, 2663, 906: 5700, 1136: 5704}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3286, 2664, 2665, 2663, 907: 5700, 1137: 5704}, {953, 953, 7: 953, 57: 953, 458: 953, 460: 953, 466: 953, 468: 953, 476: 953, 953, 479: 953, 953, 953, 483: 953, 488: 953, 490: 953}, {1811, 1811, 57: 1811, 458: 1811, 460: 1811, 466: 1811, 468: 1811, 476: 1811, 1811, 479: 1811, 1811, 1811, 483: 1811, 488: 1811, 490: 1811, 3234, 494: 3232, 3233, 3231, 3229, 501: 1811, 720: 3230, 3228}, {983, 983, 57: 983, 458: 983, 460: 983, 466: 983, 468: 983, 476: 983, 983, 479: 983, 983, 983, 483: 983, 488: 983, 490: 983, 499: 983}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 509: 5723, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 5724, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5722, 1025: 5725, 1187: 5726, 1261: 5727}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 509: 5723, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 5724, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5722, 1026: 5725, 1188: 5726, 1261: 5727}, // 3260 {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, 832, 832, 832, 832, 832, 832, 832, 58: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 459: 832, 461: 832, 832, 832, 832, 469: 832, 832, 832, 832, 832, 478: 832, 484: 832, 486: 832, 492: 832, 832, 500: 832, 509: 832, 529: 832, 552: 832, 554: 832, 832, 832, 832, 832, 832, 832, 832, 832, 564: 832, 832, 832, 832, 832, 832, 572: 832, 574: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 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, 638: 832, 732: 832, 832, 735: 832, 832, 832, 743: 832, 755: 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, 831, 831, 831, 831, 831, 831, 831, 58: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 459: 831, 461: 831, 831, 831, 831, 469: 831, 831, 831, 831, 831, 478: 831, 484: 831, 486: 831, 492: 831, 831, 500: 831, 509: 831, 529: 831, 552: 831, 554: 831, 831, 831, 831, 831, 831, 831, 831, 831, 564: 831, 831, 831, 831, 831, 831, 572: 831, 574: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 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, 638: 831, 732: 831, 831, 735: 831, 831, 831, 743: 831, 755: 831, 831, 831}, @@ -9925,13 +9925,13 @@ var ( {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, 826, 826, 826, 826, 826, 826, 826, 58: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 459: 826, 461: 826, 826, 826, 826, 469: 826, 826, 826, 826, 826, 478: 826, 484: 826, 486: 826, 492: 826, 826, 500: 826, 509: 826, 529: 826, 552: 826, 554: 826, 826, 826, 826, 826, 826, 826, 826, 826, 564: 826, 826, 826, 826, 826, 826, 572: 826, 574: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 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, 638: 826, 732: 826, 826, 735: 826, 826, 826, 743: 826, 755: 826, 826, 826}, {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, 825, 825, 825, 825, 825, 825, 825, 58: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 459: 825, 461: 825, 825, 825, 825, 469: 825, 825, 825, 825, 825, 478: 825, 484: 825, 486: 825, 492: 825, 825, 500: 825, 509: 825, 529: 825, 552: 825, 554: 825, 825, 825, 825, 825, 825, 825, 825, 825, 564: 825, 825, 825, 825, 825, 825, 572: 825, 574: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 636: 825, 638: 825, 732: 825, 825, 735: 825, 825, 825, 743: 825, 755: 825, 825, 825}, {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, 824, 824, 824, 824, 824, 824, 824, 58: 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 459: 824, 461: 824, 824, 824, 824, 469: 824, 824, 824, 824, 824, 478: 824, 484: 824, 486: 824, 492: 824, 824, 500: 824, 509: 824, 529: 824, 552: 824, 554: 824, 824, 824, 824, 824, 824, 824, 824, 824, 564: 824, 824, 824, 824, 824, 824, 572: 824, 574: 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 636: 824, 638: 824, 732: 824, 824, 735: 824, 824, 824, 743: 824, 755: 824, 824, 824}, - {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, 822, 822, 822, 822, 822, 822, 822, 58: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 5713, 5719, 5720, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 459: 822, 461: 822, 822, 822, 822, 469: 822, 822, 822, 822, 822, 478: 822, 484: 822, 486: 822, 492: 822, 822, 500: 5716, 509: 822, 529: 822, 552: 822, 554: 822, 822, 822, 822, 822, 822, 822, 822, 822, 564: 822, 822, 822, 822, 822, 822, 572: 822, 574: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 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, 638: 3432, 732: 3430, 3431, 735: 5218, 5217, 5216, 743: 5213, 755: 5712, 5715, 5711, 768: 5634, 772: 5709, 824: 5710, 847: 5708, 1099: 5721, 5714}, + {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, 822, 822, 822, 822, 822, 822, 822, 58: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 5713, 5719, 5720, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 459: 822, 461: 822, 822, 822, 822, 469: 822, 822, 822, 822, 822, 478: 822, 484: 822, 486: 822, 492: 822, 822, 500: 5716, 509: 822, 529: 822, 552: 822, 554: 822, 822, 822, 822, 822, 822, 822, 822, 822, 564: 822, 822, 822, 822, 822, 822, 572: 822, 574: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 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, 638: 3432, 732: 3430, 3431, 735: 5218, 5217, 5216, 743: 5213, 755: 5712, 5715, 5711, 768: 5634, 771: 5709, 824: 5710, 847: 5708, 1100: 5721, 5714}, // 3270 {2: 820, 820, 820, 820, 820, 8: 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 58: 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 459: 820, 461: 820, 820, 820, 820, 469: 820, 820, 820, 820, 820, 478: 820, 484: 820, 486: 820, 492: 820, 820, 500: 820, 509: 820, 529: 820, 552: 820, 554: 820, 820, 820, 820, 820, 820, 820, 820, 820, 564: 820, 820, 820, 820, 820, 820, 572: 820, 574: 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 820, 636: 820, 638: 820, 732: 820, 820, 735: 820, 820, 820, 743: 820, 755: 820, 820, 820}, {2: 816, 816, 816, 816, 816, 8: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 58: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 459: 816, 461: 816, 816, 816, 816, 469: 816, 816, 816, 816, 816, 478: 816, 484: 816, 486: 816, 492: 816, 816, 500: 816, 509: 816, 529: 816, 552: 816, 554: 816, 816, 816, 816, 816, 816, 816, 816, 816, 564: 816, 816, 816, 816, 816, 816, 572: 816, 574: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 636: 816, 638: 816, 732: 816, 816, 735: 816, 816, 816, 743: 816, 755: 816, 816, 816}, {2: 815, 815, 815, 815, 815, 8: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 58: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 459: 815, 461: 815, 815, 815, 815, 469: 815, 815, 815, 815, 815, 478: 815, 484: 815, 486: 815, 492: 815, 815, 500: 815, 509: 815, 529: 815, 552: 815, 554: 815, 815, 815, 815, 815, 815, 815, 815, 815, 564: 815, 815, 815, 815, 815, 815, 572: 815, 574: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 636: 815, 638: 815, 732: 815, 815, 735: 815, 815, 815, 743: 815, 755: 815, 815, 815}, {2: 821, 821, 821, 821, 821, 8: 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 58: 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 459: 821, 461: 821, 821, 821, 821, 469: 821, 821, 821, 821, 821, 478: 821, 484: 821, 486: 821, 492: 821, 821, 500: 821, 509: 821, 529: 821, 552: 821, 554: 821, 821, 821, 821, 821, 821, 821, 821, 821, 564: 821, 821, 821, 821, 821, 821, 572: 821, 574: 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 821, 636: 821, 638: 821, 732: 821, 821, 735: 821, 821, 821, 743: 821, 755: 821, 821, 821}, - {1821, 1821, 2900, 2748, 2784, 2902, 2675, 1821, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 1821, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 1821, 460: 1821, 5739, 465: 5738, 1821, 468: 1821, 476: 1821, 1821, 479: 1821, 1821, 1821, 483: 1821, 487: 1821, 1821, 1821, 1821, 3234, 494: 3232, 3233, 3231, 3229, 499: 1821, 648: 5737, 2664, 2665, 2663, 720: 3230, 3228, 1184: 5736, 5735}, + {1821, 1821, 2900, 2748, 2784, 2902, 2675, 1821, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 1821, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 1821, 460: 1821, 5739, 465: 5738, 1821, 468: 1821, 476: 1821, 1821, 479: 1821, 1821, 1821, 483: 1821, 487: 1821, 1821, 1821, 1821, 3234, 494: 3232, 3233, 3231, 3229, 499: 1821, 648: 5737, 2664, 2665, 2663, 720: 3230, 3228, 1185: 5736, 5735}, // 3275 {1825, 1825, 7: 1825, 57: 1825, 458: 1825, 460: 1825, 466: 1825, 468: 1825, 476: 1825, 1825, 479: 1825, 1825, 1825, 483: 1825, 487: 1825, 1825, 1825, 1825, 499: 1825}, {1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 460: 1241, 1241, 1241, 1241, 465: 1241, 1241, 1241, 1241, 471: 1241, 1241, 1241, 476: 1241, 1241, 479: 1241, 1241, 1241, 1241, 1241, 487: 1241, 1241, 1241, 1241, 1241, 494: 1241, 1241, 1241, 1241, 499: 1241, 507: 1241, 509: 1241, 530: 1241, 533: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 570: 1241, 639: 5730, 642: 1241, 1241}, @@ -9939,7 +9939,7 @@ var ( {814, 814, 7: 5728, 57: 814, 458: 814, 460: 814, 466: 814, 468: 814, 476: 814, 814, 479: 814, 814, 814, 483: 814, 487: 814, 814, 814, 814, 499: 814}, {984, 984, 57: 984, 458: 984, 460: 984, 466: 984, 468: 984, 476: 984, 984, 479: 984, 984, 984, 483: 984, 487: 984, 984, 984, 984, 499: 984}, // 3280 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 509: 5723, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 5724, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5722, 1025: 5729}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 509: 5723, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 5724, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 5722, 1026: 5729}, {1814, 1814, 7: 1814, 57: 1814, 458: 1814, 460: 1814, 466: 1814, 468: 1814, 476: 1814, 1814, 479: 1814, 1814, 1814, 483: 1814, 487: 1814, 1814, 1814, 1814, 499: 1814}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 5731, 648: 5732, 2664, 2665, 2663}, {1824, 1824, 7: 1824, 57: 1824, 458: 1824, 460: 1824, 466: 1824, 468: 1824, 476: 1824, 1824, 479: 1824, 1824, 1824, 483: 1824, 487: 1824, 1824, 1824, 1824, 499: 1824}, @@ -9988,31 +9988,31 @@ var ( {1005, 1005}, // 3320 {1004, 1004}, - {2: 1316, 1316, 1316, 1316, 1316, 8: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 58: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 481: 5770, 1043: 5771}, + {2: 1316, 1316, 1316, 1316, 1316, 8: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 58: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 481: 5770, 1044: 5771}, {2: 1315, 1315, 1315, 1315, 1315, 8: 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 58: 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315, 1315}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5772}, {150: 887, 459: 887, 887, 475: 5280, 484: 887, 498: 887, 553: 887, 632: 887, 842: 5773}, // 3325 - {150: 5781, 459: 5774, 2496, 484: 5782, 498: 5780, 553: 2494, 632: 2490, 696: 5779, 738: 5777, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5778, 5776, 3764, 957: 5775, 1042: 5783}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 2250, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 2497, 2496, 484: 2495, 553: 2494, 632: 2490, 648: 4024, 2664, 2665, 2663, 696: 5646, 729: 4025, 738: 3765, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 3767, 3766, 3764, 771: 4953, 981: 5795}, - {459: 3781, 826: 5792, 955: 5791}, + {150: 5781, 459: 5774, 2496, 484: 5782, 498: 5780, 553: 2494, 632: 2490, 696: 5779, 738: 5777, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5778, 5776, 3764, 958: 5775, 1043: 5783}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 2250, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 2497, 2496, 484: 2495, 553: 2494, 632: 2490, 648: 4024, 2664, 2665, 2663, 696: 5646, 729: 4025, 738: 3765, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 3767, 3766, 3764, 794: 4953, 982: 5795}, + {459: 3781, 826: 5792, 956: 5791}, {1308, 1308, 458: 1308, 468: 1308}, {1307, 1307, 458: 1307, 466: 781, 468: 1307, 476: 781, 781}, // 3330 {1306, 1306, 458: 1306, 468: 1306}, {1305, 1305, 458: 1305, 466: 780, 468: 1305, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, - {1291, 1291, 2900, 2748, 2784, 2902, 2675, 1291, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 1291, 468: 1291, 648: 4024, 2664, 2665, 2663, 729: 5785, 985: 5786, 1167: 5784}, + {1291, 1291, 2900, 2748, 2784, 2902, 2675, 1291, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 1291, 468: 1291, 648: 4024, 2664, 2665, 2663, 729: 5785, 986: 5786, 1168: 5784}, {459: 1303}, - {459: 1302, 561: 3780, 895: 3779, 956: 3778}, + {459: 1302, 561: 3780, 896: 3779, 957: 3778}, // 3335 {1286, 1286, 468: 1286}, {1304, 1304, 7: 5789, 458: 1304, 468: 1304}, {482: 5787}, {1290, 1290, 7: 1290, 458: 1290, 468: 1290}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 791: 5788}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 2689, 2741, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 2770, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 2668, 2684, 2827, 2918, 2775, 2702, 2719, 2846, 2929, 2762, 2731, 2840, 2841, 2836, 2796, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 2777, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 2781, 2693, 2728, 2662, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 2700, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 2766, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 2767, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 2835, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 2653, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 2783, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 2725, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 2654, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 2678, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3051, 3052, 3100, 3099, 2955, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 2817, 2834, 2956, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3079, 3080, 3090, 3076, 3077, 3078, 3108, 2778, 459: 3147, 461: 3127, 3145, 2657, 3787, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3156, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 554: 2655, 3181, 3165, 3117, 3119, 3121, 3120, 3148, 3125, 564: 3138, 3150, 3126, 3159, 3157, 3149, 572: 3154, 574: 3225, 3161, 3170, 3171, 3172, 3124, 3141, 3142, 3195, 3198, 3199, 3200, 3201, 3202, 3152, 3203, 3178, 3183, 3193, 3194, 3187, 3204, 3205, 3206, 3188, 3208, 3209, 3196, 3189, 3207, 3184, 3192, 3190, 3176, 3210, 3211, 3153, 3215, 3166, 3167, 3169, 3214, 3220, 3219, 3221, 3218, 3222, 3217, 3216, 3213, 3162, 3212, 3168, 3173, 3174, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3146, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3783, 790: 5788}, // 3340 {1292, 1292, 7: 1292, 458: 1292, 468: 1292}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5785, 985: 5790}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5785, 986: 5790}, {1289, 1289, 7: 1289, 458: 1289, 468: 1289}, {1309, 1309, 7: 5793, 458: 1309, 468: 1309}, {1301, 1301, 7: 1301, 458: 1301, 468: 1301}, @@ -10020,8 +10020,8 @@ var ( {459: 3781, 826: 5794}, {1300, 1300, 7: 1300, 458: 1300, 468: 1300}, {57: 5796}, - {150: 5781, 459: 2497, 2496, 484: 5782, 553: 2494, 632: 2490, 696: 5801, 738: 5799, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5800, 5798, 3764, 957: 5797}, - {459: 3781, 826: 5792, 955: 5802}, + {150: 5781, 459: 2497, 2496, 484: 5782, 553: 2494, 632: 2490, 696: 5801, 738: 5799, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5800, 5798, 3764, 958: 5797}, + {459: 3781, 826: 5792, 956: 5802}, // 3350 {1313, 1313, 458: 1313, 468: 1313}, {1312, 1312, 458: 1312, 466: 781, 468: 1312, 476: 781, 781}, @@ -10029,20 +10029,20 @@ var ( {1310, 1310, 458: 1310, 466: 780, 468: 1310, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, {1314, 1314, 7: 5793, 458: 1314, 468: 1314}, // 3355 - {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 474: 1020, 481: 1020, 735: 5218, 5217, 5216, 824: 5219, 865: 5804}, + {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 474: 1020, 481: 1020, 735: 5218, 5217, 5216, 824: 5219, 866: 5804}, {2: 1803, 1803, 1803, 1803, 1803, 8: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 58: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 474: 4071, 481: 1803, 838: 5805}, - {2: 1316, 1316, 1316, 1316, 1316, 8: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 58: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 481: 5770, 1043: 5806}, + {2: 1316, 1316, 1316, 1316, 1316, 8: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 58: 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 1316, 481: 5770, 1044: 5806}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5807}, {150: 887, 459: 887, 887, 475: 5280, 484: 887, 498: 887, 553: 887, 632: 887, 842: 5808}, // 3360 - {150: 5781, 459: 5774, 2496, 484: 5782, 498: 5780, 553: 2494, 632: 2490, 696: 5779, 738: 5777, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5778, 5776, 3764, 957: 5775, 1042: 5809}, + {150: 5781, 459: 5774, 2496, 484: 5782, 498: 5780, 553: 2494, 632: 2490, 696: 5779, 738: 5777, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 5778, 5776, 3764, 958: 5775, 1043: 5809}, {1288, 1288, 458: 5811, 468: 1288, 1232: 5810}, {1317, 1317, 468: 1317}, {192: 5812}, {563: 5813}, // 3365 {641: 5814}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 870: 5335, 908: 5815}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 5334, 871: 5335, 909: 5815}, {1287, 1287, 7: 5337, 468: 1287}, {1321, 1321, 459: 5824, 639: 1779}, {1322, 1322}, @@ -10067,7 +10067,7 @@ var ( // 3385 {98: 3828, 107: 3827, 110: 5934, 204: 5933, 820: 5935}, {644, 644}, - {636, 636, 176: 5915, 217: 5916, 227: 5917, 230: 5914, 248: 5919, 258: 5918, 273: 5921, 278: 5920, 479: 636, 636, 488: 636, 638: 5922, 1105: 5913, 1271: 5912, 5911}, + {636, 636, 176: 5915, 217: 5916, 227: 5917, 230: 5914, 248: 5919, 258: 5918, 273: 5921, 278: 5920, 479: 636, 636, 488: 636, 638: 5922, 1106: 5913, 1271: 5912, 5911}, {642, 642}, {641, 641}, // 3390 @@ -10081,21 +10081,21 @@ var ( {612, 612, 489: 612, 507: 612}, {611, 611, 489: 611, 507: 611}, {110: 5899}, - {487: 5875, 530: 5876, 792: 5894}, + {487: 5875, 530: 5876, 791: 5894}, // 3400 - {98: 571, 107: 571, 197: 5873, 1067: 5888}, + {98: 571, 107: 571, 197: 5873, 1068: 5888}, {603, 603, 489: 603, 507: 603}, {602, 602, 489: 602, 507: 602}, {125: 5886, 138: 5887, 189: 5885}, {598, 598, 489: 598, 507: 598}, // 3405 - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 792: 5878, 827: 5884}, + {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5884}, {125: 5883}, {125: 5882}, {125: 5881}, {125: 5880}, // 3410 - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 792: 5878, 827: 5877}, + {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5877}, {591, 591, 489: 591, 507: 591}, {590, 590, 489: 590, 507: 590}, {589, 589, 489: 589, 507: 589}, @@ -10132,19 +10132,19 @@ var ( {599, 599, 489: 599, 507: 599}, // 3440 {98: 3828, 107: 3827, 820: 5889}, - {487: 5875, 530: 5876, 792: 5891, 1107: 5890}, - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 792: 5878, 827: 5893}, + {487: 5875, 530: 5876, 791: 5891, 1108: 5890}, + {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5893}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5892}, {567, 567, 487: 567, 489: 567, 507: 567, 530: 567}, // 3445 {604, 604, 489: 604, 507: 604}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5895, 2664, 2665, 2663, 724: 5896}, - {1018, 1018, 487: 5875, 489: 1018, 507: 1018, 530: 5876, 639: 3810, 792: 5897}, + {1018, 1018, 487: 5875, 489: 1018, 507: 1018, 530: 5876, 639: 3810, 791: 5897}, {607, 607, 489: 607, 507: 607}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5898, 2664, 2665, 2663}, // 3450 {606, 606, 489: 606, 507: 606}, - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 792: 5878, 827: 5900}, + {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5900}, {609, 609, 489: 609, 507: 609}, {553: 5906, 568: 5903, 833: 5905, 1269: 5904}, {578, 578, 489: 578, 507: 578}, @@ -10176,7 +10176,7 @@ var ( {628, 628, 7: 628, 479: 628, 628, 488: 628}, {629, 629, 7: 629, 479: 629, 629, 488: 629}, {630, 630, 7: 630, 479: 630, 630, 488: 630}, - {176: 5915, 217: 5916, 227: 5917, 230: 5914, 248: 5919, 258: 5918, 273: 5921, 278: 5920, 638: 5922, 1105: 5927}, + {176: 5915, 217: 5916, 227: 5917, 230: 5914, 248: 5919, 258: 5918, 273: 5921, 278: 5920, 638: 5922, 1106: 5927}, {633, 633, 7: 633, 479: 633, 633, 488: 633}, // 3480 {834, 834, 480: 2630, 488: 2631, 754: 2632, 815: 5932}, @@ -10186,9 +10186,9 @@ var ( {643, 643}, // 3485 {645, 645}, - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 792: 5878, 827: 5938}, - {487: 5875, 530: 5876, 792: 5891, 1107: 5936}, - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 792: 5878, 827: 5937}, + {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5938}, + {487: 5875, 530: 5876, 791: 5891, 1108: 5936}, + {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5937}, {605, 605, 489: 605, 507: 605}, // 3490 {610, 610, 489: 610, 507: 610}, @@ -10197,17 +10197,17 @@ var ( {621, 621, 468: 5943, 1303: 5942}, {647, 647}, // 3495 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 5540, 5538, 844: 5944}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 5540, 5538, 844: 5944}, {620, 620, 7: 5541}, - {569, 569, 15: 1683, 153: 1683, 475: 1683, 487: 5875, 489: 569, 507: 569, 530: 5876, 635: 1683, 639: 1683, 792: 5878, 827: 5955}, + {569, 569, 15: 1683, 153: 1683, 475: 1683, 487: 5875, 489: 569, 507: 569, 530: 5876, 635: 1683, 639: 1683, 791: 5878, 827: 5955}, {15: 887, 153: 5948, 475: 5280, 635: 887, 842: 5947}, {15: 5949, 635: 5950}, // 3500 {650, 650}, - {237, 237, 489: 2624, 778: 2625, 5954}, + {237, 237, 489: 2624, 777: 2625, 5954}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5951, 2664, 2665, 2663}, {15: 5952}, - {237, 237, 489: 2624, 778: 2625, 5953}, + {237, 237, 489: 2624, 777: 2625, 5953}, // 3505 {649, 649}, {651, 651}, @@ -10223,7 +10223,7 @@ var ( // 3515 {652, 652}, {653, 653}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 864: 5966}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 865: 5966}, {654, 654}, {655, 655}, // 3520 @@ -10239,10 +10239,10 @@ var ( {576, 576, 467: 3238, 570: 3239}, {1902, 1902, 198: 5979, 553: 1902, 1235: 5978}, // 3530 - {545, 545, 553: 5981, 949: 5980}, + {545, 545, 553: 5981, 950: 5980}, {1901, 1901, 553: 1901}, {1907, 1907}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 5982}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 5982}, {544, 544, 7: 3808}, // 3535 {2: 1904, 1904, 1904, 1904, 1904, 8: 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 58: 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 1904, 552: 5985, 1201: 5984}, @@ -10261,9 +10261,9 @@ var ( {493: 2638, 722: 5995}, {1913, 1913}, {487: 6005}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 6000, 648: 5211, 2664, 2665, 2663, 801: 6002, 1176: 6001}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 6000, 648: 5211, 2664, 2665, 2663, 801: 6002, 1177: 6001}, // 3550 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 5999}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 5999}, {7: 3808, 487: 1960, 637: 1960}, {487: 1962, 637: 1962}, {7: 6003, 487: 1961, 637: 1961}, @@ -10272,8 +10272,8 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5211, 2664, 2665, 2663, 801: 6004}, {7: 1958, 487: 1958, 637: 1958}, {461: 6006}, - {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 878: 6007}, - {1963, 1963, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 458: 6025, 640: 6035, 874: 6027, 6026, 6029, 6008, 879: 6028}, + {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 879: 6007}, + {1963, 1963, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 458: 6025, 640: 6035, 875: 6027, 6026, 6029, 6008, 880: 6028}, // 3560 {1956, 1956, 27: 1956, 59: 1956, 61: 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 1956, 458: 1956, 640: 1956}, {482: 1955, 493: 1955}, @@ -10308,7 +10308,7 @@ var ( {98: 1987, 107: 1987, 482: 4171, 493: 1987, 723: 6048}, {177: 1987, 1987, 181: 1987, 482: 4171, 493: 1987, 557: 1987, 1987, 723: 6045}, {177: 1987, 1987, 181: 1987, 482: 4171, 493: 1987, 557: 1987, 1987, 723: 6036}, - {177: 6042, 6043, 181: 6044, 493: 2638, 557: 6040, 6041, 722: 6039, 910: 6037, 1070: 6038}, + {177: 6042, 6043, 181: 6044, 493: 2638, 557: 6040, 6041, 722: 6039, 911: 6037, 1071: 6038}, {1924, 1924, 27: 1924, 59: 1924, 61: 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 458: 1924, 640: 1924}, // 3590 {1923, 1923, 27: 1923, 59: 1923, 61: 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 1923, 458: 1923, 640: 1923}, @@ -10319,7 +10319,7 @@ var ( // 3595 {1915, 1915, 27: 1915, 59: 1915, 61: 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 458: 1915, 640: 1915}, {1914, 1914, 27: 1914, 59: 1914, 61: 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 458: 1914, 640: 1914}, - {177: 6042, 6043, 181: 6044, 493: 2638, 557: 6040, 6041, 722: 6039, 910: 6046, 1070: 6047}, + {177: 6042, 6043, 181: 6044, 493: 2638, 557: 6040, 6041, 722: 6039, 911: 6046, 1071: 6047}, {1926, 1926, 27: 1926, 59: 1926, 61: 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 458: 1926, 640: 1926}, {1925, 1925, 27: 1925, 59: 1925, 61: 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 458: 1925, 640: 1925}, // 3600 @@ -10337,7 +10337,7 @@ var ( // 3610 {1930, 1930, 27: 1930, 59: 1930, 61: 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 458: 1930, 640: 1930}, {461: 6061, 493: 2638, 722: 2637, 730: 6060}, - {1932, 1932, 27: 1932, 59: 1932, 61: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 458: 1932, 640: 1932, 793: 6062}, + {1932, 1932, 27: 1932, 59: 1932, 61: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 105: 3350, 3346, 108: 3343, 3358, 111: 3345, 3342, 3344, 3348, 3349, 3354, 3353, 3352, 3356, 3357, 3351, 3355, 3347, 458: 1932, 640: 1932, 792: 6062}, {1933, 1933, 27: 1933, 59: 1933, 61: 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 458: 1933, 640: 1933}, {292: 6063}, // 3615 @@ -10350,7 +10350,7 @@ var ( {244, 244, 27: 244, 59: 244, 61: 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 458: 244, 640: 244}, {461: 6070}, {1936, 1936, 27: 1936, 59: 1936, 61: 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 458: 1936, 640: 1936}, - {493: 2638, 557: 6040, 6041, 722: 6039, 910: 6072}, + {493: 2638, 557: 6040, 6041, 722: 6039, 911: 6072}, {1937, 1937, 27: 1937, 59: 1937, 61: 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 458: 1937, 640: 1937}, // 3625 {493: 2638, 722: 2637, 730: 6074}, @@ -10359,19 +10359,19 @@ var ( {637: 6077}, {461: 6078}, // 3630 - {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 878: 6079}, - {1964, 1964, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 458: 6025, 640: 6035, 874: 6027, 6026, 6029, 6008, 879: 6028}, + {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 879: 6079}, + {1964, 1964, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 458: 6025, 640: 6035, 875: 6027, 6026, 6029, 6008, 880: 6028}, {1979, 1979, 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 6105}, {1977, 1977}, {37: 6103}, // 3635 {1716, 1716, 1716, 1716, 1716, 1716, 1716, 8: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 58: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 482: 6086, 639: 1716}, - {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 774: 2615, 776: 2616, 2614, 790: 2466, 795: 2613, 808: 2619, 835: 6085}, + {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 6085}, {1971, 1971}, - {174: 6090, 298: 6093, 314: 6092, 385: 6089, 390: 6094, 461: 6087, 561: 6091, 1181: 6088}, - {459: 2497, 2496, 479: 6099, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 774: 2615, 776: 2616, 2614, 790: 2466, 795: 2613, 808: 2619, 835: 6100}, + {174: 6090, 298: 6093, 314: 6092, 385: 6089, 390: 6094, 461: 6087, 561: 6091, 1182: 6088}, + {459: 2497, 2496, 479: 6099, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 6100}, // 3640 - {459: 2497, 2496, 479: 6095, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 774: 2615, 776: 2616, 2614, 790: 2466, 795: 2613, 808: 2619, 835: 6096}, + {459: 2497, 2496, 479: 6095, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 653: 2610, 696: 2611, 731: 2464, 738: 2612, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2618, 2617, 2467, 766: 2593, 2465, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 6096}, {459: 1970, 1970, 479: 1970, 484: 1970, 492: 1970, 553: 1970, 555: 1970, 632: 1970, 641: 1970, 653: 1970, 731: 1970}, {459: 1969, 1969, 479: 1969, 484: 1969, 492: 1969, 553: 1969, 555: 1969, 632: 1969, 641: 1969, 653: 1969, 731: 1969}, {459: 1968, 1968, 479: 1968, 484: 1968, 492: 1968, 553: 1968, 555: 1968, 632: 1968, 641: 1968, 653: 1968, 731: 1968}, @@ -10397,7 +10397,7 @@ var ( // 3660 {449, 449, 466: 780, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, {451, 451, 466: 781, 476: 781, 781}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 5498, 5503, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 5501, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 5500, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 5505, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5499, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 5506, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 5502, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 486: 3721, 554: 5512, 574: 5511, 634: 3719, 648: 5509, 2664, 2665, 2663, 759: 5513, 817: 5510, 958: 5514, 1134: 5507}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 5498, 5503, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 5501, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 5500, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 5505, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 5499, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 5506, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 5502, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 486: 3721, 554: 5512, 574: 5511, 634: 3719, 648: 5509, 2664, 2665, 2663, 759: 5513, 817: 5510, 959: 5514, 1135: 5507}, {456, 456}, {455, 455}, // 3665 @@ -10414,79 +10414,79 @@ var ( {32: 5015}, // 3675 {461: 6124}, - {82: 2458, 159: 2460, 165: 2486, 168: 2457, 459: 2497, 2496, 484: 2495, 492: 2481, 498: 6110, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6108, 731: 2464, 738: 6109, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6116, 6115, 2467, 766: 2593, 2465, 774: 6113, 776: 6114, 6112, 790: 2466, 795: 6111, 814: 6122, 872: 6118, 882: 6119, 887: 6117, 894: 6120, 897: 6121, 1127: 6125}, + {82: 2458, 159: 2460, 165: 2486, 168: 2457, 459: 2497, 2496, 484: 2495, 492: 2481, 498: 6110, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6108, 731: 2464, 738: 6109, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6116, 6115, 2467, 766: 2593, 2465, 773: 6113, 775: 6114, 6112, 789: 2466, 795: 6111, 814: 6122, 873: 6118, 883: 6119, 888: 6117, 895: 6120, 898: 6121, 1128: 6125}, {1983, 1983}, {2010, 2010}, {2009, 2009}, // 3680 {242, 242, 468: 242}, - {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 474: 1020, 487: 1020, 735: 5218, 5217, 5216, 824: 5219, 865: 6130}, - {2: 1008, 1008, 1008, 1008, 1008, 8: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 58: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 6132, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 474: 1008, 487: 1008, 1085: 6131}, + {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 474: 1020, 487: 1020, 735: 5218, 5217, 5216, 824: 5219, 866: 6130}, + {2: 1008, 1008, 1008, 1008, 1008, 8: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 58: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 6132, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 474: 1008, 487: 1008, 1086: 6131}, {2: 1803, 1803, 1803, 1803, 1803, 8: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 58: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 474: 4071, 487: 1803, 838: 6133}, {2: 1007, 1007, 1007, 1007, 1007, 8: 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 58: 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 1007, 474: 1007, 487: 1007}, // 3685 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 487: 6134, 648: 6136, 2664, 2665, 2663, 901: 6137, 947: 6135}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6151, 2664, 2665, 2663, 724: 6149, 901: 6137, 947: 6150}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 487: 6134, 648: 6136, 2664, 2665, 2663, 902: 6137, 948: 6135}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6151, 2664, 2665, 2663, 724: 6149, 902: 6137, 948: 6150}, {7: 6145, 487: 6144}, - {7: 1010, 468: 1010, 487: 1010, 639: 6139, 889: 6138}, + {7: 1010, 468: 1010, 487: 1010, 639: 6139, 890: 6138}, {7: 1012, 468: 1012, 487: 1012}, // 3690 {7: 1014, 468: 1014, 487: 1014}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 6141, 648: 6140, 2664, 2665, 2663}, - {7: 1010, 468: 1010, 487: 1010, 639: 6143, 889: 6142}, + {7: 1010, 468: 1010, 487: 1010, 639: 6143, 890: 6142}, {7: 1009, 468: 1009, 487: 1009}, {7: 1013, 468: 1013, 487: 1013}, // 3695 {509: 6141}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5232, 834: 5222, 868: 6147}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6136, 2664, 2665, 2663, 901: 6146}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5232, 834: 5222, 869: 6147}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6136, 2664, 2665, 2663, 902: 6146}, {7: 1011, 468: 1011, 487: 1011}, - {237, 237, 7: 5276, 468: 237, 489: 2624, 778: 2625, 6148}, + {237, 237, 7: 5276, 468: 237, 489: 2624, 777: 2625, 6148}, // 3700 {2014, 2014, 468: 2014}, {887, 887, 887, 887, 887, 887, 887, 8: 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 58: 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 465: 887, 468: 887, 474: 887, 5280, 480: 887, 485: 887, 489: 887, 887, 511: 887, 842: 6157}, {7: 6145, 468: 6154}, - {1018, 1018, 1018, 1018, 1018, 1018, 1018, 1010, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 58: 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 465: 1018, 468: 1010, 474: 1018, 1018, 480: 1018, 485: 1018, 489: 1018, 1018, 511: 1018, 639: 6152, 889: 6138}, + {1018, 1018, 1018, 1018, 1018, 1018, 1018, 1010, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 58: 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 465: 1018, 468: 1010, 474: 1018, 1018, 480: 1018, 485: 1018, 489: 1018, 1018, 511: 1018, 639: 6152, 890: 6138}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 509: 6141, 648: 6153, 2664, 2665, 2663}, // 3705 - {1017, 1017, 1017, 1017, 1017, 1017, 1017, 1010, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 58: 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 465: 1017, 468: 1010, 474: 1017, 1017, 480: 1017, 485: 1017, 489: 1017, 1017, 511: 1017, 639: 6143, 889: 6142}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 784: 5227, 787: 5226, 5232, 834: 5222, 868: 6155}, - {237, 237, 7: 5276, 489: 2624, 778: 2625, 6156}, + {1017, 1017, 1017, 1017, 1017, 1017, 1017, 1010, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 58: 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 465: 1017, 468: 1010, 474: 1017, 1017, 480: 1017, 485: 1017, 489: 1017, 1017, 511: 1017, 639: 6143, 890: 6142}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 5230, 565: 5225, 648: 3805, 2664, 2665, 2663, 696: 5229, 724: 5228, 783: 5227, 786: 5226, 5232, 834: 5222, 869: 6155}, + {237, 237, 7: 5276, 489: 2624, 777: 2625, 6156}, {2013, 2013}, - {885, 885, 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 465: 5237, 468: 885, 474: 885, 480: 885, 485: 885, 489: 885, 885, 511: 885, 648: 5236, 2664, 2665, 2663, 899: 5235, 6158}, + {885, 885, 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 465: 5237, 468: 885, 474: 885, 480: 885, 485: 885, 489: 885, 885, 511: 885, 648: 5236, 2664, 2665, 2663, 900: 5235, 6158}, // 3710 - {866, 866, 468: 866, 474: 5290, 480: 866, 485: 5291, 489: 866, 866, 511: 5289, 923: 5293, 5292, 1039: 5294, 6159}, - {237, 237, 468: 237, 480: 237, 489: 2624, 237, 778: 2625, 6160}, + {866, 866, 468: 866, 474: 5290, 480: 866, 485: 5291, 489: 866, 866, 511: 5289, 924: 5293, 5292, 1040: 5294, 6159}, + {237, 237, 468: 237, 480: 237, 489: 2624, 237, 777: 2625, 6160}, {1258, 1258, 468: 1258, 480: 1258, 490: 2627, 753: 2628, 797: 6161}, - {848, 848, 468: 848, 480: 5340, 1048: 6162}, + {848, 848, 468: 848, 480: 5340, 1049: 6162}, {2015, 2015, 468: 2015}, // 3715 {2016, 2016, 7: 3476}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6237, 2664, 2665, 2663}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6235}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6226}, - {110: 5354, 553: 5353, 1123: 6222}, + {110: 5354, 553: 5353, 1124: 6222}, // 3720 {151: 573, 156: 5403}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 552: 6217, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 6216}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 552: 6217, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 6216}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 552: 6213, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5113, 848: 6212}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 552: 6209, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 5540, 5538, 844: 6208}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 552: 6209, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 5540, 5538, 844: 6208}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6204}, // 3725 {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6202}, {151: 6182}, {154: 6179}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6177}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 6178}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 6178}, // 3730 {26, 26, 7: 3808}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6180}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 864: 6181}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 865: 6181}, {53, 53}, {479: 6183}, // 3735 - {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 774: 6192, 776: 6193, 6190, 873: 6194}, - {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 474: 818, 487: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 902: 6197}, + {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 773: 6192, 775: 6193, 6190, 874: 6194}, + {2: 818, 818, 818, 818, 818, 8: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 58: 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, 474: 818, 487: 818, 735: 818, 818, 818, 743: 5213, 847: 5214, 903: 6197}, {459: 2497, 484: 2495, 553: 2494, 632: 2490, 641: 2594, 696: 3772, 738: 3771, 2491, 2492, 2493, 2502, 744: 2500, 3773, 3774, 766: 6128}, {175, 175, 466: 780, 468: 175, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, {177, 177, 466: 781, 468: 177, 476: 781, 781}, @@ -10499,50 +10499,50 @@ var ( // 3745 {171, 171, 468: 171}, {169, 169, 468: 6195}, - {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 774: 6192, 776: 6193, 6190, 873: 6196}, + {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 773: 6192, 775: 6193, 6190, 874: 6196}, {168, 168}, - {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 474: 1020, 487: 1020, 735: 5218, 5217, 5216, 824: 5219, 865: 6198}, + {2: 1020, 1020, 1020, 1020, 1020, 8: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 58: 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 1020, 474: 1020, 487: 1020, 735: 5218, 5217, 5216, 824: 5219, 866: 6198}, // 3750 - {2: 1008, 1008, 1008, 1008, 1008, 8: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 58: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 6132, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 474: 1008, 487: 1008, 1085: 6199}, + {2: 1008, 1008, 1008, 1008, 1008, 8: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 58: 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 6132, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 1008, 474: 1008, 487: 1008, 1086: 6199}, {2: 1803, 1803, 1803, 1803, 1803, 8: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 58: 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 1803, 474: 4071, 487: 1803, 838: 6200}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 487: 6201, 648: 6136, 2664, 2665, 2663, 901: 6137, 947: 6135}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 487: 6201, 648: 6136, 2664, 2665, 2663, 902: 6137, 948: 6135}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6149}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6203, 2664, 2665, 2663}, // 3755 {1908, 1908}, {1995, 1995, 160: 6206, 475: 6205}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 775: 6207}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 774: 6207}, {1993, 1993}, {1994, 1994, 7: 4626}, // 3760 {1997, 1997, 7: 5541}, {569: 6210}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 5540, 5538, 844: 6211}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 5540, 5538, 844: 6211}, {1996, 1996, 7: 5541}, {1999, 1999, 7: 5115}, // 3765 {569: 6214}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5113, 848: 6215}, {1998, 1998, 7: 5115}, - {1992, 1992, 7: 3808, 657: 4677, 659: 4676, 893: 6221}, + {1992, 1992, 7: 3808, 657: 4677, 659: 4676, 894: 6221}, {569: 6218}, // 3770 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 6219}, - {1992, 1992, 7: 3808, 657: 4677, 659: 4676, 893: 6220}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 6219}, + {1992, 1992, 7: 3808, 657: 4677, 659: 4676, 894: 6220}, {2000, 2000}, {2001, 2001}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6223}, // 3775 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 783: 6224}, - {1992, 1992, 7: 3808, 657: 4677, 659: 4676, 893: 6225}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 3806, 782: 6224}, + {1992, 1992, 7: 3808, 657: 4677, 659: 4676, 894: 6225}, {2005, 2005}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6227, 2664, 2665, 2663}, {458: 6228}, // 3780 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6229}, - {2134, 2134, 92: 4125, 483: 4126, 850: 6231, 861: 6230, 1041: 6232}, + {2134, 2134, 92: 4125, 483: 4126, 850: 6231, 862: 6230, 1042: 6232}, {2133, 2133, 92: 4125, 850: 6234}, - {2132, 2132, 483: 4126, 861: 6233}, + {2132, 2132, 483: 4126, 862: 6233}, {2006, 2006}, // 3785 {2130, 2130}, @@ -10569,9 +10569,9 @@ var ( {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 552: 4701, 769: 6252}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6253}, // 3805 - {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, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 464: 49, 466: 49, 49, 485: 49, 49, 634: 49, 49, 644: 6262, 896: 6256, 1171: 6254, 1264: 6255}, - {385, 385, 4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 464: 4130, 466: 4167, 2106, 485: 4844, 2106, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 780: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 903: 5425, 997: 6287}, - {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, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 464: 48, 466: 48, 48, 485: 48, 48, 634: 48, 48, 644: 6262, 896: 6286}, + {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, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 464: 49, 466: 49, 49, 485: 49, 49, 634: 49, 49, 644: 6262, 897: 6256, 1172: 6254, 1264: 6255}, + {385, 385, 4: 4131, 4133, 389, 13: 2106, 4150, 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 4089, 4148, 4168, 4152, 4139, 4132, 4135, 4134, 4137, 4138, 4140, 4147, 389, 4158, 4159, 4145, 4146, 4151, 4153, 4165, 4164, 4170, 4166, 4163, 4156, 4161, 4162, 4155, 4157, 4160, 4149, 464: 4130, 466: 4167, 2106, 485: 4844, 2106, 634: 2106, 4136, 758: 4087, 762: 4088, 764: 4141, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 904: 5425, 998: 6287}, + {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, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 464: 48, 466: 48, 48, 485: 48, 48, 634: 48, 48, 644: 6262, 897: 6286}, {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, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 81: 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 464: 47, 466: 47, 47, 485: 47, 47, 634: 47, 47, 644: 47}, {471: 1987, 1987, 482: 4171, 493: 1987, 646: 6283, 723: 6282}, // 3810 @@ -10612,8 +10612,8 @@ var ( {50, 50}, // 3840 {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 552: 4701, 769: 6289}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 864: 6290}, - {15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6291, 1080: 6292}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 865: 6290}, + {15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6291, 1081: 6292}, {2443, 2443, 7: 2443, 15: 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443}, {52, 52, 7: 6294, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6293}, // 3845 @@ -10621,25 +10621,25 @@ var ( {15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6295}, {2441, 2441, 7: 2441, 15: 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441}, {479: 6297}, - {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 774: 6192, 776: 6193, 6190, 873: 6298}, + {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 773: 6192, 775: 6193, 6190, 874: 6298}, // 3850 {468: 6299}, - {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 774: 6192, 776: 6193, 6190, 873: 6300}, + {459: 2497, 2496, 484: 2495, 492: 2481, 553: 2494, 555: 2480, 632: 2490, 641: 2594, 696: 6186, 731: 6184, 738: 6187, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6189, 6188, 6185, 766: 2593, 6191, 773: 6192, 775: 6193, 6190, 874: 6300}, {170, 170}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 6303, 5538, 1098: 6304, 1259: 6302}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 6303, 5538, 1099: 6304, 1259: 6302}, {232, 232, 7: 6305}, // 3855 {181, 181, 7: 181}, {180, 180, 7: 180}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 786: 5539, 806: 6303, 5538, 1098: 6306}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 5532, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 5537, 648: 3377, 2664, 2665, 2663, 725: 5066, 785: 5539, 806: 6303, 5538, 1099: 6306}, {179, 179, 7: 179}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 869: 5130, 905: 6308}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 870: 5130, 906: 6308}, // 3860 - {215, 215, 7: 5132, 14: 215, 58: 215, 460: 215, 652: 5176, 940: 5175, 6309}, - {223, 223, 14: 223, 58: 223, 460: 6311, 988: 6310}, - {202, 202, 14: 6328, 58: 6326, 933: 6327, 6325, 1078: 6324, 6323}, - {126: 6316, 6314, 6315, 6317, 987: 6313, 1169: 6312}, - {222, 222, 14: 222, 58: 222, 126: 6316, 6314, 6315, 6317, 987: 6322}, + {215, 215, 7: 5132, 14: 215, 58: 215, 460: 215, 652: 5176, 941: 5175, 6309}, + {223, 223, 14: 223, 58: 223, 460: 6311, 989: 6310}, + {202, 202, 14: 6328, 58: 6326, 934: 6327, 6325, 1079: 6324, 6323}, + {126: 6316, 6314, 6315, 6317, 988: 6313, 1170: 6312}, + {222, 222, 14: 222, 58: 222, 126: 6316, 6314, 6315, 6317, 988: 6322}, // 3865 {221, 221, 14: 221, 58: 221, 126: 221, 221, 221, 221}, {493: 2638, 722: 3934, 748: 6321}, @@ -10654,13 +10654,13 @@ var ( {220, 220, 14: 220, 58: 220, 126: 220, 220, 220, 220}, // 3875 {233, 233}, - {201, 201, 14: 6328, 58: 6326, 933: 6327, 6338}, + {201, 201, 14: 6328, 58: 6326, 934: 6327, 6338}, {200, 200, 14: 200, 58: 200}, - {483: 6337, 953: 6336}, + {483: 6337, 954: 6336}, {196, 196, 14: 196, 58: 196, 202: 6332, 464: 6333, 567: 6331}, // 3880 {320: 6329}, - {191, 191, 14: 191, 58: 191, 202: 191, 464: 191, 567: 191, 1161: 6330}, + {191, 191, 14: 191, 58: 191, 202: 191, 464: 191, 567: 191, 1162: 6330}, {192, 192, 14: 192, 58: 192, 202: 192, 464: 192, 567: 192}, {493: 2638, 722: 3934, 748: 6334}, {194, 194, 14: 194, 58: 194}, @@ -10675,9 +10675,9 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6340, 2664, 2665, 2663}, {487: 6341}, {461: 6342}, - {1900, 1900, 27: 1900, 59: 1900, 61: 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 137: 6345, 458: 1900, 492: 6344, 640: 1900, 1021: 6343}, + {1900, 1900, 27: 1900, 59: 1900, 61: 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 137: 6345, 458: 1900, 492: 6344, 640: 1900, 1022: 6343}, // 3895 - {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 878: 6350}, + {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 879: 6350}, {1899, 1899, 27: 1899, 59: 1899, 61: 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 458: 1899, 640: 1899}, {192: 6348, 374: 6349, 630: 6347, 638: 6346}, {1898, 1898, 27: 1898, 59: 1898, 61: 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 458: 1898, 640: 1898}, @@ -10685,7 +10685,7 @@ var ( // 3900 {1896, 1896, 27: 1896, 59: 1896, 61: 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 458: 1896, 640: 1896}, {1895, 1895, 27: 1895, 59: 1895, 61: 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 458: 1895, 640: 1895}, - {1912, 1912, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 458: 6025, 640: 6035, 874: 6027, 6026, 6029, 6008, 879: 6028}, + {1912, 1912, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 458: 6025, 640: 6035, 875: 6027, 6026, 6029, 6008, 880: 6028}, {92: 2034, 99: 2034, 139: 2034, 655: 2034}, {99: 2029, 139: 6359, 655: 2029, 1308: 6358}, // 3905 @@ -10710,14 +10710,14 @@ var ( {459: 6371, 465: 2023, 1309: 6370}, {459: 2024, 465: 2024}, {465: 6377}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6373, 2664, 2665, 2663, 1163: 6372}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6373, 2664, 2665, 2663, 1164: 6372}, {7: 6375, 57: 6374}, // 3925 {7: 2021, 57: 2021}, {465: 2022}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6376, 2664, 2665, 2663}, {7: 2020, 57: 2020}, - {459: 2497, 2496, 484: 2495, 553: 2494, 632: 2490, 696: 6381, 738: 6379, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6380, 6378, 3764, 1173: 6382}, + {459: 2497, 2496, 484: 2495, 553: 2494, 632: 2490, 696: 6381, 738: 6379, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 6380, 6378, 3764, 1174: 6382}, // 3930 {2042, 2042, 460: 2042}, {2041, 2041, 460: 2041, 466: 781, 476: 781, 781}, @@ -10729,18 +10729,18 @@ var ( {136: 6386, 302: 6385}, {571: 6389}, {571: 6387}, - {888: 6388}, + {889: 6388}, // 3940 {2017, 2017}, - {888: 6390}, + {889: 6390}, {2018, 2018}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5211, 2664, 2665, 2663, 801: 6392}, - {2115, 2115, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 780: 6394, 832: 6397, 915: 6396, 1174: 6393}, + {2115, 2115, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 779: 6394, 832: 6397, 916: 6396, 1175: 6393}, // 3945 {2123, 2123}, {13: 3720, 26: 4089, 28: 6401, 467: 6400, 486: 3721, 634: 3719, 759: 6399, 762: 6402}, {2116, 2116, 13: 2116, 15: 2116, 2116, 2116, 2116, 2116, 2116, 2116, 2116, 2116, 2116, 2116, 2116, 28: 2116, 464: 2116, 467: 2116, 486: 2116, 634: 2116}, - {2114, 2114, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 780: 6394, 832: 6398}, + {2114, 2114, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 779: 6394, 832: 6398}, {2113, 2113, 13: 2113, 15: 2113, 2113, 2113, 2113, 2113, 2113, 2113, 2113, 2113, 2113, 2113, 2113, 28: 2113, 464: 2113, 467: 2113, 486: 2113, 634: 2113}, // 3950 {2112, 2112, 13: 2112, 15: 2112, 2112, 2112, 2112, 2112, 2112, 2112, 2112, 2112, 2112, 2112, 2112, 28: 2112, 464: 2112, 467: 2112, 486: 2112, 634: 2112}, @@ -10749,11 +10749,11 @@ var ( {461: 1987, 482: 4171, 723: 6403}, {2117, 2117, 13: 2117, 15: 2117, 2117, 2117, 2117, 2117, 2117, 2117, 2117, 2117, 2117, 2117, 2117, 28: 2117, 464: 2117, 467: 2117, 486: 2117, 634: 2117}, // 3955 - {461: 4189, 1019: 6404}, + {461: 4189, 1020: 6404}, {2118, 2118, 13: 2118, 15: 2118, 2118, 2118, 2118, 2118, 2118, 2118, 2118, 2118, 2118, 2118, 2118, 28: 2118, 464: 2118, 467: 2118, 486: 2118, 634: 2118}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3375, 648: 3377, 2664, 2665, 2663, 725: 3374, 852: 6406}, {2119, 2119, 13: 2119, 15: 2119, 2119, 2119, 2119, 2119, 2119, 2119, 2119, 2119, 2119, 2119, 2119, 28: 2119, 464: 2119, 467: 2119, 486: 2119, 634: 2119}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 794: 6408}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 529: 3645, 648: 3377, 2664, 2665, 2663, 725: 3644, 793: 6408}, // 3960 {2120, 2120, 13: 2120, 15: 2120, 2120, 2120, 2120, 2120, 2120, 2120, 2120, 2120, 2120, 2120, 2120, 28: 2120, 464: 2120, 467: 2120, 486: 2120, 634: 2120}, {2: 1805, 1805, 1805, 1805, 1805, 8: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 58: 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 1805, 552: 4701, 769: 6410}, @@ -10767,19 +10767,19 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 459: 4507, 648: 4024, 2664, 2665, 2663, 729: 4506, 812: 4505, 822: 6417}, {7: 4516, 57: 6418}, // 3970 - {1799, 1799, 4: 1799, 29: 1799, 92: 1799, 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 483: 1799, 858: 6419}, - {2134, 2134, 4: 4730, 29: 4727, 92: 4125, 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 483: 4126, 839: 4731, 841: 4728, 849: 4732, 6231, 857: 4726, 861: 6230, 1041: 6420}, + {1799, 1799, 4: 1799, 29: 1799, 92: 1799, 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 483: 1799, 859: 6419}, + {2134, 2134, 4: 4730, 29: 4727, 92: 4125, 4734, 4594, 4319, 4595, 4318, 460: 4729, 468: 4733, 483: 4126, 839: 4731, 841: 4728, 849: 4732, 6231, 858: 4726, 862: 6230, 1042: 6420}, {2141, 2141}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6422, 2664, 2665, 2663}, {459: 6423}, // 3975 - {220: 4763, 229: 4765, 232: 4764, 1115: 6424}, + {220: 4763, 229: 4765, 232: 4764, 1116: 6424}, {57: 6425}, {458: 6426}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6427}, {459: 6428}, // 3980 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 6429}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 794: 6429}, {7: 4027, 57: 6430}, {2143, 2143}, {2235, 2235}, @@ -10795,64 +10795,64 @@ var ( {2263, 2263}, {2264, 2264}, {2265, 2265}, - {2262, 2262, 660: 5286, 907: 6443}, + {2262, 2262, 660: 5286, 908: 6443}, // 3995 {2261, 2261}, {2268, 2268}, {2267, 2267}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6459, 783: 6458}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6459, 782: 6458}, {553: 6448}, // 4000 {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6449}, {475: 6451, 635: 6450}, {873, 873, 2900, 2748, 2784, 2902, 2675, 873, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 460: 873, 573: 4905, 648: 4904, 2664, 2665, 2663, 840: 6456}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 775: 6452}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 774: 6452}, {7: 4626, 635: 6453}, // 4005 {873, 873, 2900, 2748, 2784, 2902, 2675, 873, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 460: 873, 573: 4905, 648: 4904, 2664, 2665, 2663, 840: 6454}, - {2284, 2284, 7: 4907, 460: 4888, 789: 6455}, + {2284, 2284, 7: 4907, 460: 4888, 788: 6455}, {2292, 2292}, - {2284, 2284, 7: 4907, 460: 4888, 789: 6457}, + {2284, 2284, 7: 4907, 460: 4888, 788: 6457}, {2295, 2295}, // 4010 - {2287, 2287, 7: 3808, 155: 6479, 460: 2287, 638: 6478, 961: 6489}, + {2287, 2287, 7: 3808, 155: 6479, 460: 2287, 638: 6478, 962: 6489}, {1016, 1016, 7: 1016, 98: 6464, 155: 1016, 460: 1016, 475: 6461, 635: 6460, 638: 1016, 641: 6462, 656: 6463}, {873, 873, 2900, 2748, 2784, 2902, 2675, 873, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 460: 873, 573: 4905, 648: 4904, 2664, 2665, 2663, 840: 6487}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 775: 6474}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4625, 2664, 2665, 2663, 774: 6474}, {244: 6470}, // 4015 {244: 6467}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 6465}, - {2284, 2284, 7: 4027, 460: 4888, 789: 6466}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5656, 2664, 2665, 2663, 856: 6465}, + {2284, 2284, 7: 5658, 460: 4888, 788: 6466}, {2289, 2289}, {458: 6468}, // 4020 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 6469}, - {2290, 2290, 7: 4027}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5656, 2664, 2665, 2663, 856: 6469}, + {2290, 2290, 7: 5658}, {458: 6471}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 6472}, - {2284, 2284, 7: 4027, 460: 4888, 789: 6473}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5656, 2664, 2665, 2663, 856: 6472}, + {2284, 2284, 7: 5658, 460: 4888, 788: 6473}, // 4025 {2291, 2291}, - {2287, 2287, 7: 4626, 98: 6477, 155: 6479, 460: 2287, 635: 6476, 638: 6478, 961: 6475}, - {2284, 2284, 460: 4888, 789: 6486}, + {2287, 2287, 7: 4626, 98: 6477, 155: 6479, 460: 2287, 635: 6476, 638: 6478, 962: 6475}, + {2284, 2284, 460: 4888, 788: 6486}, {873, 873, 2900, 2748, 2784, 2902, 2675, 873, 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 460: 873, 573: 4905, 648: 4904, 2664, 2665, 2663, 840: 6484}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4024, 2664, 2665, 2663, 729: 4025, 771: 6482}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 5656, 2664, 2665, 2663, 856: 6482}, // 4030 {98: 6481}, {98: 6480}, {2285, 2285, 460: 2285}, {2286, 2286, 460: 2286}, - {2284, 2284, 7: 4027, 460: 4888, 789: 6483}, + {2284, 2284, 7: 5658, 460: 4888, 788: 6483}, // 4035 {2288, 2288}, - {2284, 2284, 7: 4907, 460: 4888, 789: 6485}, + {2284, 2284, 7: 4907, 460: 4888, 788: 6485}, {2293, 2293}, {2294, 2294}, - {2284, 2284, 7: 4907, 460: 4888, 789: 6488}, + {2284, 2284, 7: 4907, 460: 4888, 788: 6488}, // 4040 {2296, 2296}, - {2284, 2284, 460: 4888, 789: 6490}, + {2284, 2284, 460: 4888, 788: 6490}, {2297, 2297}, {553: 6496}, {479: 6494}, @@ -10863,11 +10863,11 @@ var ( {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6497}, {475: 5280, 541: 887, 635: 887, 646: 887, 842: 6498}, // 4050 - {541: 6501, 635: 6500, 646: 6502, 1110: 6499}, + {541: 6501, 635: 6500, 646: 6502, 1111: 6499}, {2305, 2305}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6509, 2664, 2665, 2663}, {459: 3781, 826: 6504}, - {459: 3781, 826: 5792, 955: 6503}, + {459: 3781, 826: 5792, 956: 6503}, // 4055 {2302, 2302, 7: 5793}, {491: 6505}, @@ -10876,10 +10876,10 @@ var ( {493: 2638, 722: 3934, 748: 6508}, // 4060 {2303, 2303}, - {541: 6501, 646: 6502, 1110: 6510}, + {541: 6501, 646: 6502, 1111: 6510}, {2304, 2304}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6512}, - {2307, 2307, 637: 6514, 1190: 6513}, + {2307, 2307, 637: 6514, 1191: 6513}, // 4065 {2308, 2308}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6515, 2664, 2665, 2663}, @@ -10891,16 +10891,16 @@ var ( {2309, 2309}, {493: 2638, 722: 3934, 748: 6521}, {2311, 2311}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6533, 1125: 6532, 1296: 6531}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6533, 1126: 6532, 1296: 6531}, // 4075 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 6526, 1131: 6525, 1301: 6524}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 6526, 1132: 6525, 1301: 6524}, {2315, 2315, 7: 6529}, {2314, 2314, 7: 2314}, {637: 6527}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 6528}, // 4080 {2312, 2312, 7: 2312}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 6526, 1131: 6530}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 6526, 1132: 6530}, {2313, 2313, 7: 2313}, {2319, 2319, 7: 6536}, {2318, 2318, 7: 2318}, @@ -10908,10 +10908,10 @@ var ( {637: 6534}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6535}, {2316, 2316, 7: 2316}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6533, 1125: 6537}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6533, 1126: 6537}, {2317, 2317, 7: 2317}, // 4090 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 6587, 6592, 6594, 6588, 6593, 6596, 6590, 6586, 6591, 6595, 6589, 2106, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 648: 5211, 2664, 2665, 2663, 758: 4087, 762: 4088, 764: 6395, 780: 6394, 801: 6598, 832: 6397, 915: 6599}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 6587, 6592, 6594, 6588, 6593, 6596, 6590, 6586, 6591, 6595, 6589, 2106, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 648: 5211, 2664, 2665, 2663, 758: 4087, 762: 4088, 764: 6395, 779: 6394, 801: 6598, 832: 6397, 916: 6599}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 6577, 2664, 2665, 2663}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 461: 1807, 552: 4341, 556: 1807, 763: 6566}, {264: 6560, 1210: 6559}, @@ -10919,8 +10919,8 @@ var ( // 4095 {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6544}, {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 6545}, - {81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 6549, 644: 6262, 896: 6548, 967: 6547, 1144: 6546}, - {25, 25, 81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 6549, 644: 6262, 896: 6548, 967: 6554}, + {81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 6549, 644: 6262, 897: 6548, 968: 6547, 1145: 6546}, + {25, 25, 81: 6261, 6258, 6264, 6265, 6266, 6259, 6257, 6267, 6263, 6260, 6549, 644: 6262, 897: 6548, 968: 6554}, {24, 24, 81: 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 644: 24}, // 4100 {22, 22, 81: 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 644: 22}, @@ -10932,8 +10932,8 @@ var ( {20, 20, 81: 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 644: 20}, {23, 23, 81: 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 644: 23}, {2: 1807, 1807, 1807, 1807, 1807, 8: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 58: 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 1807, 552: 4341, 763: 6556}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 864: 6557}, - {15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6291, 1080: 6558}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 4861, 2664, 2665, 2663, 865: 6557}, + {15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6291, 1081: 6558}, // 4110 {51, 51, 7: 6294, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6293}, {229, 229}, @@ -10944,28 +10944,28 @@ var ( {458: 6564}, {195: 6565}, {227, 227}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 6567, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 869: 5130, 905: 6568}, + {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 6567, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5129, 870: 5130, 906: 6568}, {1660, 1660, 7: 1660, 14: 1660, 58: 1660, 141: 1660, 459: 6572, 1660, 554: 1660, 652: 1660, 654: 1660}, // 4120 - {215, 215, 7: 5132, 14: 215, 58: 215, 460: 215, 652: 5176, 940: 5175, 6569}, - {223, 223, 14: 223, 58: 223, 460: 6311, 988: 6570}, - {202, 202, 14: 6328, 58: 6326, 933: 6327, 6325, 1078: 6324, 6571}, + {215, 215, 7: 5132, 14: 215, 58: 215, 460: 215, 652: 5176, 941: 5175, 6569}, + {223, 223, 14: 223, 58: 223, 460: 6311, 989: 6570}, + {202, 202, 14: 6328, 58: 6326, 934: 6327, 6325, 1079: 6324, 6571}, {231, 231}, {57: 6573}, // 4125 {141: 6574}, {646: 6575}, - {461: 5145, 871: 6576}, + {461: 5145, 872: 6576}, {230, 230}, - {1900, 1900, 27: 1900, 59: 1900, 61: 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 137: 6345, 458: 1900, 492: 6344, 640: 1900, 1021: 6578}, + {1900, 1900, 27: 1900, 59: 1900, 61: 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 137: 6345, 458: 1900, 492: 6344, 640: 1900, 1022: 6578}, // 4130 - {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 878: 6579}, - {1894, 1894, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 6581, 458: 6025, 640: 6035, 874: 6027, 6026, 6029, 6008, 879: 6028, 1204: 6580}, + {1957, 1957, 27: 1957, 59: 1957, 61: 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 1957, 458: 1957, 640: 1957, 879: 6579}, + {1894, 1894, 27: 6034, 59: 6010, 61: 6030, 6023, 6013, 6009, 6017, 6021, 6033, 6016, 6022, 6020, 6018, 6031, 6024, 6012, 6032, 6011, 6014, 6015, 6019, 6581, 458: 6025, 640: 6035, 875: 6027, 6026, 6029, 6008, 880: 6028, 1204: 6580}, {1909, 1909}, {198: 6583, 638: 6582}, - {545, 545, 553: 5981, 949: 6585}, + {545, 545, 553: 5981, 950: 6585}, // 4135 - {545, 545, 553: 5981, 949: 6584}, + {545, 545, 553: 5981, 950: 6584}, {1892, 1892}, {1893, 1893}, {13: 1328, 15: 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 28: 1328, 461: 1987, 464: 1328, 467: 1328, 482: 4171, 486: 1328, 634: 1328, 723: 4882}, @@ -10983,10 +10983,10 @@ var ( {13: 1324, 15: 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 28: 1324, 461: 1987, 464: 1324, 467: 1324, 482: 4171, 486: 1324, 634: 1324, 723: 4862}, {13: 1377, 15: 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 28: 1377, 154: 4854, 464: 1377, 467: 1377, 486: 1377, 634: 1377}, // 4150 - {13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 780: 6394, 832: 6397, 915: 6600}, - {2124, 2124, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 780: 6394, 832: 6398}, - {2125, 2125, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 780: 6394, 832: 6398}, - {1985, 1985, 59: 2477, 80: 2592, 82: 2458, 91: 2488, 159: 2460, 163: 2482, 165: 2486, 168: 2457, 196: 2507, 205: 2453, 214: 2506, 2473, 2459, 231: 2485, 236: 2463, 239: 2483, 241: 2454, 243: 2489, 259: 2604, 261: 2475, 265: 2474, 272: 2487, 274: 2455, 277: 2476, 288: 2468, 459: 2497, 2496, 483: 2600, 2495, 492: 2481, 498: 2505, 511: 2595, 515: 2471, 553: 2494, 555: 2480, 632: 2490, 635: 2603, 640: 2456, 2594, 653: 2451, 656: 2462, 661: 2461, 666: 2504, 673: 2452, 696: 2501, 731: 2464, 738: 2503, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2574, 2573, 2467, 766: 2593, 2465, 774: 2557, 776: 2568, 2584, 790: 2466, 795: 2523, 808: 2511, 814: 2598, 837: 2596, 846: 2478, 872: 2518, 882: 2521, 887: 2560, 894: 2565, 897: 2575, 914: 2530, 918: 2469, 953: 2599, 960: 2509, 962: 2510, 2513, 2514, 966: 2516, 968: 2515, 970: 2512, 972: 2517, 2519, 2520, 976: 2479, 2556, 979: 2526, 989: 2534, 2527, 2528, 2529, 2535, 2533, 2536, 2537, 998: 2532, 2531, 1001: 2522, 2484, 2470, 2538, 2550, 2539, 2540, 2541, 2543, 2547, 2544, 2548, 2549, 2542, 2546, 2545, 1018: 2508, 1022: 2524, 2525, 2472, 1028: 2552, 2551, 1032: 2554, 2555, 2553, 1037: 2590, 2558, 1045: 2602, 2601, 2559, 1052: 2561, 1054: 2587, 1081: 2562, 2563, 1084: 2564, 1086: 2569, 1089: 2566, 2567, 1092: 2589, 2570, 2597, 2572, 2571, 1102: 2577, 2576, 2580, 1106: 2581, 1108: 2588, 1111: 2578, 6602, 1116: 2579, 1128: 2582, 2583, 2586, 1132: 2585}, + {13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 779: 6394, 832: 6397, 916: 6600}, + {2124, 2124, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 779: 6394, 832: 6398}, + {2125, 2125, 13: 2106, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 2106, 28: 2106, 464: 4130, 467: 2106, 486: 2106, 634: 2106, 758: 4087, 762: 4088, 764: 6395, 779: 6394, 832: 6398}, + {1985, 1985, 59: 2477, 80: 2592, 82: 2458, 91: 2488, 159: 2460, 163: 2482, 165: 2486, 168: 2457, 196: 2507, 205: 2453, 214: 2506, 2473, 2459, 231: 2485, 236: 2463, 239: 2483, 241: 2454, 243: 2489, 259: 2604, 261: 2475, 265: 2474, 272: 2487, 274: 2455, 277: 2476, 288: 2468, 459: 2497, 2496, 483: 2600, 2495, 492: 2481, 498: 2505, 511: 2595, 515: 2471, 553: 2494, 555: 2480, 632: 2490, 635: 2603, 640: 2456, 2594, 653: 2451, 656: 2462, 661: 2461, 666: 2504, 673: 2452, 696: 2501, 731: 2464, 738: 2503, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 2574, 2573, 2467, 766: 2593, 2465, 773: 2557, 775: 2568, 2584, 789: 2466, 795: 2523, 808: 2511, 814: 2598, 837: 2596, 846: 2478, 873: 2518, 883: 2521, 888: 2560, 895: 2565, 898: 2575, 915: 2530, 919: 2469, 954: 2599, 961: 2509, 963: 2510, 2513, 2514, 967: 2516, 969: 2515, 971: 2512, 973: 2517, 2519, 2520, 977: 2479, 2556, 980: 2526, 990: 2534, 2527, 2528, 2529, 2535, 2533, 2536, 2537, 999: 2532, 2531, 1002: 2522, 2484, 2470, 2538, 2550, 2539, 2540, 2541, 2543, 2547, 2544, 2548, 2549, 2542, 2546, 2545, 1019: 2508, 1023: 2524, 2525, 2472, 1029: 2552, 2551, 1033: 2554, 2555, 2553, 1038: 2590, 2558, 1046: 2602, 2601, 2559, 1053: 2561, 1055: 2587, 1082: 2562, 2563, 1085: 2564, 1087: 2569, 1090: 2566, 2567, 1093: 2589, 2570, 2597, 2572, 2571, 1103: 2577, 2576, 2580, 1107: 2581, 1109: 2588, 1112: 2578, 6602, 1117: 2579, 1129: 2582, 2583, 2586, 1133: 2585}, {433, 433}, } ) @@ -12202,7 +12202,7 @@ yynewstate: { parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-5].item.(*ast.TableName)}, - ColumnNames: yyS[yypt-1].item.([]*ast.ColumnName), + ColumnNames: yyS[yypt-1].item.([]model.CIStr), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt), HistogramOperation: ast.HistogramOperationUpdate, } @@ -12211,7 +12211,7 @@ yynewstate: { parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-4].item.(*ast.TableName)}, - ColumnNames: yyS[yypt-0].item.([]*ast.ColumnName), + ColumnNames: yyS[yypt-0].item.([]model.CIStr), HistogramOperation: ast.HistogramOperationDrop, } } @@ -12219,7 +12219,7 @@ yynewstate: { parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-3].item.(*ast.TableName)}, - ColumnNames: yyS[yypt-1].item.([]*ast.ColumnName), + ColumnNames: yyS[yypt-1].item.([]model.CIStr), ColumnChoice: model.ColumnList, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } @@ -12228,7 +12228,7 @@ yynewstate: parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-5].item.(*ast.TableName)}, PartitionNames: yyS[yypt-3].item.([]model.CIStr), - ColumnNames: yyS[yypt-1].item.([]*ast.ColumnName), + ColumnNames: yyS[yypt-1].item.([]model.CIStr), ColumnChoice: model.ColumnList, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } diff --git a/parser/parser.y b/parser/parser.y index 99854380c3990..6d6507b5ad604 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -2597,37 +2597,37 @@ AnalyzeTableStmt: AnalyzeOpts: $9.([]ast.AnalyzeOpt), } } -| "ANALYZE" "TABLE" TableName "UPDATE" "HISTOGRAM" "ON" ColumnNameList AnalyzeOptionListOpt +| "ANALYZE" "TABLE" TableName "UPDATE" "HISTOGRAM" "ON" IdentList AnalyzeOptionListOpt { $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$3.(*ast.TableName)}, - ColumnNames: $7.([]*ast.ColumnName), + ColumnNames: $7.([]model.CIStr), AnalyzeOpts: $8.([]ast.AnalyzeOpt), HistogramOperation: ast.HistogramOperationUpdate, } } -| "ANALYZE" "TABLE" TableName "DROP" "HISTOGRAM" "ON" ColumnNameList +| "ANALYZE" "TABLE" TableName "DROP" "HISTOGRAM" "ON" IdentList { $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$3.(*ast.TableName)}, - ColumnNames: $7.([]*ast.ColumnName), + ColumnNames: $7.([]model.CIStr), HistogramOperation: ast.HistogramOperationDrop, } } -| "ANALYZE" "TABLE" TableName "COLUMNS" ColumnNameList AnalyzeOptionListOpt +| "ANALYZE" "TABLE" TableName "COLUMNS" IdentList AnalyzeOptionListOpt { $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$3.(*ast.TableName)}, - ColumnNames: $5.([]*ast.ColumnName), + ColumnNames: $5.([]model.CIStr), ColumnChoice: model.ColumnList, AnalyzeOpts: $6.([]ast.AnalyzeOpt)} } -| "ANALYZE" "TABLE" TableName "PARTITION" PartitionNameList "COLUMNS" ColumnNameList AnalyzeOptionListOpt +| "ANALYZE" "TABLE" TableName "PARTITION" PartitionNameList "COLUMNS" IdentList AnalyzeOptionListOpt { $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$3.(*ast.TableName)}, PartitionNames: $5.([]model.CIStr), - ColumnNames: $7.([]*ast.ColumnName), + ColumnNames: $7.([]model.CIStr), ColumnChoice: model.ColumnList, AnalyzeOpts: $8.([]ast.AnalyzeOpt)} } diff --git a/parser/parser_test.go b/parser/parser_test.go index aa116398ce475..acea251d3ac30 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -5161,6 +5161,8 @@ func TestAnalyze(t *testing.T) { {"analyze table t drop histogram on b", true, "ANALYZE TABLE `t` DROP HISTOGRAM ON `b`"}, {"analyze table t update histogram on c1, c2;", true, "ANALYZE TABLE `t` UPDATE HISTOGRAM ON `c1`,`c2`"}, {"analyze table t drop histogram on c1, c2;", true, "ANALYZE TABLE `t` DROP HISTOGRAM ON `c1`,`c2`"}, + {"analyze table t update histogram on t.c1, t.c2", false, ""}, + {"analyze table t drop histogram on t.c1, t.c2", false, ""}, {"analyze table t1,t2 all columns", true, "ANALYZE TABLE `t1`,`t2` ALL COLUMNS"}, {"analyze table t partition a all columns", true, "ANALYZE TABLE `t` PARTITION `a` ALL COLUMNS"}, {"analyze table t1,t2 all columns with 4 topn", true, "ANALYZE TABLE `t1`,`t2` ALL COLUMNS WITH 4 TOPN"}, @@ -5171,6 +5173,8 @@ func TestAnalyze(t *testing.T) { {"analyze table t partition a predicate columns with 1024 buckets", true, "ANALYZE TABLE `t` PARTITION `a` PREDICATE COLUMNS WITH 1024 BUCKETS"}, {"analyze table t columns c1,c2", true, "ANALYZE TABLE `t` COLUMNS `c1`,`c2`"}, {"analyze table t partition a columns c1,c2", true, "ANALYZE TABLE `t` PARTITION `a` COLUMNS `c1`,`c2`"}, + {"analyze table t columns t.c1,t.c2", false, ""}, + {"analyze table t partition a columns t.c1,t.c2", false, ""}, {"analyze table t columns c1,c2 with 4 topn", true, "ANALYZE TABLE `t` COLUMNS `c1`,`c2` WITH 4 TOPN"}, {"analyze table t partition a columns c1,c2 with 1024 buckets", true, "ANALYZE TABLE `t` PARTITION `a` COLUMNS `c1`,`c2` WITH 1024 BUCKETS"}, {"analyze table t index a columns c", false, ""}, diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 7f47741af6ced..2fa457c7e4c45 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1477,8 +1477,16 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo // If the PointGetPlan needs to read data using unique index (double read), we // can't use max uint64, because using math.MaxUint64 can't guarantee repeatable-read // and the data and index would be inconsistent! - isPointGet := v.IndexInfo == nil || (v.IndexInfo.Primary && v.TblInfo.IsCommonHandle) - return isPointGet, nil + // If the PointGetPlan needs to read data from Cache Table, we can't use max uint64, + // because math.MaxUint64 always make cacheData invalid. + noSecondRead := v.IndexInfo == nil || (v.IndexInfo.Primary && v.TblInfo.IsCommonHandle) + if !noSecondRead { + return false, nil + } + if v.TblInfo != nil && (v.TblInfo.TableCacheStatusType != model.TableCacheStatusDisable) { + return false, nil + } + return true, nil default: return false, nil } diff --git a/planner/core/errors.go b/planner/core/errors.go index a0b3933ab5f87..6670d64d0b1fd 100644 --- a/planner/core/errors.go +++ b/planner/core/errors.go @@ -107,4 +107,5 @@ var ( // ErrPartitionNoTemporary returns when partition at temporary mode ErrPartitionNoTemporary = dbterror.ClassOptimizer.NewStd(mysql.ErrPartitionNoTemporary) ErrViewSelectTemporaryTable = dbterror.ClassOptimizer.NewStd(mysql.ErrViewSelectTmptable) + ErrSubqueryMoreThan1Row = dbterror.ClassOptimizer.NewStd(mysql.ErrSubqueryNo1Row) ) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 575a177697325..d09f5a00b76d5 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -51,6 +51,7 @@ func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) us := PhysicalUnionScan{ Conditions: p.conditions, HandleCols: p.handleCols, + CacheTable: p.cacheTable, }.Init(p.ctx, p.stats, p.blockOffset, childProp) return []PhysicalPlan{us}, true, nil } diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go index 3df48371d9779..a26c85cbce50d 100644 --- a/planner/core/expression_rewriter_test.go +++ b/planner/core/expression_rewriter_test.go @@ -16,6 +16,7 @@ package core_test import ( . "github.com/pingcap/check" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/util/collate" @@ -466,6 +467,31 @@ func (s *testExpressionRewriterSuiteSerial) TestBetweenExprCollation(c *C) { 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() + }() + + tk.MustExec("use test") + tk.MustExec("DROP TABLE if exists t1, t2, source;") + tk.MustExec("CREATE TABLE t1(a INTEGER PRIMARY KEY);") + tk.MustExec("CREATE TABLE t2(a INTEGER);") + tk.MustExec("CREATE TABLE source (b INTEGER);") + tk.MustExec("INSERT INTO t1 VALUES (1);") + tk.MustExec("INSERT INTO t2 VALUES (1);") + tk.MustExec("INSERT INTO source VALUES (1),(1);") + // the on duplicate is not triggered by t1's primary key. + tk.MustGetErrCode("INSERT INTO t1 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);", mysql.ErrSubqueryNo1Row) + // the on duplicate is not triggered. + tk.MustExec("INSERT INTO t2 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);") + tk.MustExec("DROP TABLE if exists t1, t2, source;") +} + func (s *testExpressionRewriterSuite) TestMultiColInExpression(c *C) { store, dom, err := newStoreWithBootstrap() c.Assert(err, IsNil) diff --git a/planner/core/indexmerge_test.go b/planner/core/indexmerge_test.go index c69215b52a173..13827af5d1f23 100644 --- a/planner/core/indexmerge_test.go +++ b/planner/core/indexmerge_test.go @@ -16,42 +16,17 @@ package core import ( "context" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/hint" - "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testIndexMergeSuite{}) - -type testIndexMergeSuite struct { - *parser.Parser - - is infoschema.InfoSchema - ctx sessionctx.Context - - testdata testutil.TestData -} - -func (s *testIndexMergeSuite) SetUpSuite(c *C) { - s.is = infoschema.MockInfoSchema([]*model.TableInfo{MockSignedTable(), MockView()}) - s.ctx = MockContext() - s.Parser = parser.New() - var err error - s.testdata, err = testutil.LoadTestSuiteData("testdata", "index_merge_suite") - c.Assert(err, IsNil) -} - -func (s *testIndexMergeSuite) TearDownSuite(c *C) { - c.Assert(s.testdata.GenerateOutputIfNeeded(), IsNil) -} - func getIndexMergePathDigest(paths []*util.AccessPath, startIndex int) string { if len(paths) == startIndex { return "[]" @@ -82,31 +57,34 @@ func getIndexMergePathDigest(paths []*util.AccessPath, startIndex int) string { return idxMergeDisgest } -func (s *testIndexMergeSuite) TestIndexMergePathGeneration(c *C) { - defer testleak.AfterTest(c)() +func TestIndexMergePathGeneration(t *testing.T) { + t.Parallel() var input, output []string - s.testdata.GetTestCases(c, &input, &output) + indexMergeSuiteData.GetTestCases(t, &input, &output) ctx := context.TODO() + sctx := MockContext() + is := infoschema.MockInfoSchema([]*model.TableInfo{MockSignedTable(), MockView()}) + + parser := parser.New() + for i, tc := range input { - comment := Commentf("case:%v sql:%s", i, tc) - stmt, err := s.ParseOneStmt(tc, "", "") - c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - c.Assert(err, IsNil) - builder, _ := NewPlanBuilder().Init(MockContext(), s.is, &hint.BlockHintProcessor{}) + stmt, err := parser.ParseOneStmt(tc, "", "") + require.NoErrorf(t, err, "case:%v sql:%s", i, tc) + err = Preprocess(sctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) + builder, _ := NewPlanBuilder().Init(MockContext(), is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) if err != nil { - s.testdata.OnRecord(func() { + testdata.OnRecord(func() { output[i] = err.Error() }) - c.Assert(err.Error(), Equals, output[i], comment) + require.Equal(t, output[i], err.Error(), "case:%v sql:%s", i, tc) continue } - c.Assert(err, IsNil) + require.NoError(t, err) p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - c.Assert(err, IsNil) + require.NoError(t, err) lp := p.(LogicalPlan) - c.Assert(err, IsNil) var ds *DataSource for ds == nil { switch v := lp.(type) { @@ -119,11 +97,11 @@ func (s *testIndexMergeSuite) TestIndexMergePathGeneration(c *C) { ds.ctx.GetSessionVars().SetEnableIndexMerge(true) idxMergeStartIndex := len(ds.possibleAccessPaths) _, err = lp.recursiveDeriveStats(nil) - c.Assert(err, IsNil) + require.NoError(t, err) result := getIndexMergePathDigest(ds.possibleAccessPaths, idxMergeStartIndex) - s.testdata.OnRecord(func() { + testdata.OnRecord(func() { output[i] = result }) - c.Assert(result, Equals, output[i], comment) + require.Equalf(t, output[i], result, "case:%v sql:%s", i, tc) } } diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 63b3a44f2e3dc..1a2a3eb67b4d3 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -4642,8 +4642,9 @@ func (s *testIntegrationSerialSuite) TestPushDownGroupConcatToTiFlash(c *C) { var input []string var output []struct { - SQL string - Plan []string + SQL string + Plan []string + Warning []string } s.testData.GetTestCases(c, &input, &output) for i, tt := range input { @@ -4653,6 +4654,26 @@ func (s *testIntegrationSerialSuite) TestPushDownGroupConcatToTiFlash(c *C) { }) 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() { + if len(warnings) > 0 { + output[i].Warning = make([]string, len(warnings)) + for j, warning := range warnings { + output[i].Warning[j] = warning.Err.Error() + } + } + }) + if len(output[i].Warning) == 0 { + c.Assert(len(warnings), Equals, 0, comment) + } else { + c.Assert(len(warnings), Equals, 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) + } + } } } @@ -4750,3 +4771,39 @@ func (s *testIntegrationSerialSuite) TestRejectSortForMPP(c *C) { res.Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testIntegrationSuite) TestIssues29711(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists tbl_29711") + tk.MustExec("CREATE TABLE `tbl_29711` (" + + "`col_250` text COLLATE utf8_unicode_ci NOT NULL," + + "`col_251` enum('Alice','Bob','Charlie','David') COLLATE utf8_unicode_ci NOT NULL DEFAULT 'Charlie'," + + "PRIMARY KEY (`col_251`,`col_250`(1)) NONCLUSTERED);") + tk.MustQuery("explain format=brief " + + "select col_250,col_251 from tbl_29711 where col_251 between 'Bob' and 'David' order by col_250,col_251 limit 6;"). + Check(testkit.Rows( + "TopN 6.00 root test.tbl_29711.col_250, test.tbl_29711.col_251, offset:0, count:6", + "└─IndexLookUp 6.00 root ", + " ├─IndexRangeScan(Build) 30.00 cop[tikv] table:tbl_29711, index:PRIMARY(col_251, col_250) range:[\"Bob\",\"Bob\"], [\"Charlie\",\"Charlie\"], [\"David\",\"David\"], keep order:false, stats:pseudo", + " └─TopN(Probe) 6.00 cop[tikv] test.tbl_29711.col_250, test.tbl_29711.col_251, offset:0, count:6", + " └─TableRowIDScan 30.00 cop[tikv] table:tbl_29711 keep order:false, stats:pseudo", + )) + + tk.MustExec("drop table if exists t29711") + tk.MustExec("CREATE TABLE `t29711` (" + + "`a` varchar(10) DEFAULT NULL," + + "`b` int(11) DEFAULT NULL," + + "`c` int(11) DEFAULT NULL," + + "KEY `ia` (`a`(2)))") + tk.MustQuery("explain format=brief select * from t29711 use index (ia) order by a limit 10;"). + Check(testkit.Rows( + "TopN 10.00 root test.t29711.a, offset:0, count:10", + "└─IndexLookUp 10.00 root ", + " ├─IndexFullScan(Build) 10000.00 cop[tikv] table:t29711, index:ia(a) keep order:false, stats:pseudo", + " └─TopN(Probe) 10.00 cop[tikv] test.t29711.a, offset:0, count:10", + " └─TableRowIDScan 10000.00 cop[tikv] table:t29711 keep order:false, stats:pseudo", + )) + +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 9fcd85a6800d3..7e8bd7ba6688c 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4150,11 +4150,11 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if err != nil { return nil, err } - // Use the txn of the transaction to determine whether the cache can be read. - buffer, cond := cachedTable.TryGetMemcache(txn.StartTS()) - if cond { - b.ctx.GetSessionVars().StmtCtx.StoreCacheTable(tbl.Meta().ID, buffer) - us := LogicalUnionScan{handleCols: handleCols}.Init(b.ctx, b.getSelectOffset()) + // Use the TS of the transaction to determine whether the cache can be used. + cacheData := cachedTable.TryReadFromCache(txn.StartTS()) + if cacheData != nil { + sessionVars.StmtCtx.ReadFromTableCache = true + us := LogicalUnionScan{handleCols: handleCols, cacheTable: cacheData}.Init(b.ctx, b.getSelectOffset()) us.SetChildren(ds) result = us } else { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index a0bb8a8070726..c800f5c577a4f 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -2071,3 +2071,58 @@ func (s *testPlanSuite) TestWindowLogicalPlanAmbiguous(c *C) { } } } + +func (s *testPlanSuite) TestLogicalOptimizeWithTraceEnabled(c *C) { + sql := "select * from t where a in (1,2)" + defer testleak.AfterTest(c)() + tt := []struct { + flags []uint64 + steps int + }{ + { + flags: []uint64{ + flagEliminateAgg, + flagPushDownAgg}, + steps: 2, + }, + { + flags: []uint64{ + flagEliminateAgg, + flagPushDownAgg, + flagPrunColumns, + flagBuildKeyInfo, + }, + steps: 4, + }, + { + flags: []uint64{}, + steps: 0, + }, + } + + for i, tc := range tt { + comment := Commentf("case:%v sql:%s", i, 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) + sctx := MockContext() + sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + ctx := context.TODO() + p, err := builder.Build(ctx, stmt) + c.Assert(err, IsNil) + flag := uint64(0) + for _, f := range tc.flags { + flag = flag | f + } + p, err = logicalOptimize(ctx, flag, p.(LogicalPlan)) + c.Assert(err, IsNil) + _, ok := p.(*LogicalProjection) + c.Assert(ok, IsTrue) + otrace := sctx.GetSessionVars().StmtCtx.LogicalOptimizeTrace + c.Assert(otrace, NotNil) + c.Assert(len(otrace.Steps), Equals, tc.steps) + } +} diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 173de775558bb..2258e2b4de31d 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" @@ -178,7 +179,7 @@ func (p *LogicalJoin) GetJoinKeys() (leftKeys, rightKeys []*expression.Column, i // the join keys of EqualConditions func (p *LogicalJoin) GetPotentialPartitionKeys() (leftKeys, rightKeys []*property.MPPPartitionColumn) { for _, expr := range p.EqualConditions { - _, coll := expr.CharsetAndCollation(p.ctx) + _, coll := expr.CharsetAndCollation() collateID := property.GetCollateIDByNameForPartition(coll) leftKeys = append(leftKeys, &property.MPPPartitionColumn{Col: expr.GetArgs()[0].(*expression.Column), CollateID: collateID}) rightKeys = append(rightKeys, &property.MPPPartitionColumn{Col: expr.GetArgs()[1].(*expression.Column), CollateID: collateID}) @@ -514,6 +515,9 @@ type LogicalUnionScan struct { conditions []expression.Expression handleCols HandleCols + + // cacheTable not nil means it's reading from cached table. + cacheTable kv.MemBuffer } // DataSource represents a tableScan without condition push down. diff --git a/planner/core/main_test.go b/planner/core/main_test.go index df250425aaaf4..236d154ff8934 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -24,7 +24,8 @@ import ( "go.uber.org/goleak" ) -var testDataMap = make(testdata.BookKeeper, 1) +var testDataMap = make(testdata.BookKeeper, 2) +var indexMergeSuiteData testdata.TestData func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() @@ -32,6 +33,8 @@ func TestMain(m *testing.M) { flag.Parse() testDataMap.LoadTestSuiteData("testdata", "integration_partition_suite") + testDataMap.LoadTestSuiteData("testdata", "index_merge_suite") + indexMergeSuiteData = testDataMap["index_merge_suite"] opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 7cc352c9b8447..55127627b7477 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/types" utilhint "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/set" + "github.com/pingcap/tidb/util/tracing" "go.uber.org/atomic" ) @@ -83,9 +84,44 @@ var optRuleList = []logicalOptRule{ &columnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver } +type logicalOptimizeOp struct { + // tracer is goring to track optimize steps during rule optimizing + tracer *tracing.LogicalOptimizeTracer +} + +func defaultLogicalOptimizeOption() *logicalOptimizeOp { + return &logicalOptimizeOp{} +} + +func (op *logicalOptimizeOp) withEnableOptimizeTracer(tracer *tracing.LogicalOptimizeTracer) *logicalOptimizeOp { + op.tracer = tracer + return op +} + +func (op *logicalOptimizeOp) appendBeforeRuleOptimize(name string, before LogicalPlan) { + if op.tracer == nil { + return + } + op.tracer.AppendRuleTracerBeforeRuleOptimize(name, before.buildLogicalPlanTrace()) +} + +func (op *logicalOptimizeOp) appendStepToCurrent(id int, tp, reason, action string) { + if op.tracer == nil { + return + } + op.tracer.AppendRuleTracerStepToCurrent(id, tp, reason, action) +} + +func (op *logicalOptimizeOp) trackAfterRuleOptimize(after LogicalPlan) { + if op.tracer == nil { + return + } + op.tracer.TrackLogicalPlanAfterRuleOptimize(after.buildLogicalPlanTrace()) +} + // logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc. type logicalOptRule interface { - optimize(context.Context, LogicalPlan) (LogicalPlan, error) + optimize(context.Context, LogicalPlan, *logicalOptimizeOp) (LogicalPlan, error) name() string } @@ -335,6 +371,15 @@ func enableParallelApply(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPla } func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (LogicalPlan, error) { + opt := defaultLogicalOptimizeOption() + stmtCtx := logic.SCtx().GetSessionVars().StmtCtx + if stmtCtx.EnableOptimizeTrace { + tracer := &tracing.LogicalOptimizeTracer{Steps: make([]*tracing.LogicalRuleOptimizeTracer, 0)} + opt = opt.withEnableOptimizeTracer(tracer) + defer func() { + stmtCtx.LogicalOptimizeTrace = tracer + }() + } var err error for i, rule := range optRuleList { // The order of flags is same as the order of optRule in the list. @@ -343,10 +388,12 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic if flag&(1< 0 diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 3b7be4fbee6be..793dbc0d31cbb 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -27,7 +27,7 @@ import ( type ppdSolver struct{} -func (s *ppdSolver) optimize(ctx context.Context, lp LogicalPlan) (LogicalPlan, error) { +func (s *ppdSolver) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { _, p := lp.PredicatePushDown(nil) return p, nil } diff --git a/planner/core/rule_result_reorder.go b/planner/core/rule_result_reorder.go index 3a4069e9945fc..7ea7d73556b4d 100644 --- a/planner/core/rule_result_reorder.go +++ b/planner/core/rule_result_reorder.go @@ -37,7 +37,7 @@ import ( type resultReorder struct { } -func (rs *resultReorder) optimize(ctx context.Context, lp LogicalPlan) (LogicalPlan, error) { +func (rs *resultReorder) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { ordered := rs.completeSort(lp) if !ordered { lp = rs.injectSort(lp) diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index 60a8fb92b39af..e6234bbc3f3dc 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -26,7 +26,7 @@ import ( type pushDownTopNOptimizer struct { } -func (s *pushDownTopNOptimizer) optimize(ctx context.Context, p LogicalPlan) (LogicalPlan, error) { +func (s *pushDownTopNOptimizer) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { return p.pushDownTopN(nil), nil } diff --git a/planner/core/stats.go b/planner/core/stats.go index ef2d8c258feae..7f0d62da1dc49 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -431,8 +431,8 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * } } } - cond, _ := ds.ctx.GetSessionVars().StmtCtx.GetCacheTable(ds.tableInfo.ID) - if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && isReadOnlyTxn && ds.tableInfo.TempTableType != model.TempTableLocal && !cond { + readFromTableCache := ds.ctx.GetSessionVars().StmtCtx.ReadFromTableCache + if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && isReadOnlyTxn && ds.tableInfo.TempTableType != model.TempTableLocal && !readFromTableCache { err := ds.generateAndPruneIndexMergePath(ds.indexMergeHints != nil) if err != nil { return nil, err diff --git a/planner/core/task.go b/planner/core/task.go index f3d0d509288b9..a6af6c136a9d4 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1295,6 +1295,23 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { return topN } +// canPushToIndexPlan checks if this TopN can be pushed to the index side of copTask. +// It can be pushed to the index side when all columns used by ByItems are available from the index side and +// there's no prefix index column. +func (p *PhysicalTopN) canPushToIndexPlan(indexPlan PhysicalPlan, byItemCols []*expression.Column) bool { + schema := indexPlan.Schema() + for _, col := range byItemCols { + pos := schema.ColumnIndex(col) + if pos == -1 { + return false + } + if schema.Columns[pos].IsPrefix { + return false + } + } + return true +} + func (p *PhysicalTopN) attach2Task(tasks ...task) task { t := tasks[0].copy() inputCount := t.count() @@ -1307,7 +1324,7 @@ func (p *PhysicalTopN) attach2Task(tasks ...task) task { // If all columns in topN are from index plan, we push it to index plan, otherwise we finish the index plan and // push it to table plan. var pushedDownTopN *PhysicalTopN - if !copTask.indexPlanFinished && len(copTask.indexPlan.Schema().ColumnsIndices(cols)) > 0 { + if !copTask.indexPlanFinished && p.canPushToIndexPlan(copTask.indexPlan, cols) { pushedDownTopN = p.getPushedDownTopN(copTask.indexPlan) copTask.indexPlan = pushedDownTopN } else { @@ -1458,6 +1475,18 @@ func CheckAggCanPushCop(sctx sessionctx.Context, aggFuncs []*aggregation.AggFunc ret = false break } + orderBySize := len(aggFunc.OrderByItems) + if orderBySize > 0 { + exprs := make([]expression.Expression, 0, orderBySize) + for _, item := range aggFunc.OrderByItems { + exprs = append(exprs, item.Expr) + } + if !expression.CanExprsPushDownWithExtraInfo(sc, exprs, client, storeType, false) { + reason = "arguments of AggFunc `" + aggFunc.Name + "` contains unsupported exprs in order-by clause" + ret = false + break + } + } pb := aggregation.AggFuncToPBExpr(sctx, client, aggFunc) if pb == nil { reason = "AggFunc `" + aggFunc.Name + "` can not be converted to pb expr" diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 8a94b10cccc6c..7cbe788e3e58f 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -385,7 +385,8 @@ "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>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" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index deb6021d19ecf..3fc9530ce6942 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -2943,6 +2943,9 @@ " └─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" ] }, { @@ -2957,6 +2960,9 @@ " └─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" ] }, { @@ -2970,6 +2976,9 @@ " └─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" ] }, { @@ -2984,6 +2993,9 @@ " └─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" ] }, { @@ -2997,6 +3009,9 @@ " └─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" ] }, { @@ -3011,6 +3026,9 @@ " └─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" ] }, { @@ -3025,6 +3043,9 @@ " └─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" ] }, { @@ -3039,6 +3060,9 @@ " └─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" ] }, { @@ -3052,6 +3076,9 @@ " └─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" ] }, { @@ -3066,6 +3093,9 @@ " └─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" ] }, { @@ -3079,6 +3109,9 @@ " └─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" ] }, { @@ -3093,6 +3126,9 @@ " └─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" ] }, { @@ -3106,6 +3142,9 @@ " └─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" ] }, { @@ -3121,6 +3160,9 @@ " └─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(15,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" ] }, { @@ -3134,6 +3176,9 @@ " └─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" ] }, { @@ -3149,6 +3194,9 @@ " └─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(15,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" ] }, { @@ -3163,6 +3211,9 @@ " └─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(15,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" ] }, { @@ -3178,6 +3229,9 @@ " └─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(15,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" ] }, { @@ -3189,6 +3243,9 @@ " └─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(15,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" ] }, { @@ -3204,6 +3261,9 @@ " └─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(15,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" ] }, { @@ -3217,6 +3277,9 @@ " └─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" ] }, { @@ -3230,6 +3293,9 @@ " └─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" ] }, { @@ -3243,6 +3309,9 @@ " └─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" ] }, { @@ -3256,6 +3325,9 @@ " └─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" ] }, { @@ -3271,6 +3343,9 @@ " └─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(15,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" ] }, { @@ -3286,6 +3361,9 @@ " └─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(15,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" ] }, { @@ -3300,6 +3378,12 @@ " └─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'" ] }, { @@ -3314,6 +3398,9 @@ " └─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" ] }, { @@ -3328,6 +3415,9 @@ " └─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" ] }, { @@ -3342,6 +3432,9 @@ " └─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" ] }, { @@ -3356,6 +3449,9 @@ " └─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" ] }, { @@ -3370,6 +3466,9 @@ " └─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" ] }, { @@ -3384,6 +3483,17 @@ " └─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'" ] }, { @@ -3397,6 +3507,9 @@ " └─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" ] }, { @@ -3410,6 +3523,9 @@ " └─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" ] }, { @@ -3423,6 +3539,9 @@ " └─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" ] }, { @@ -3437,6 +3556,12 @@ " └─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'" ] }, { @@ -3450,6 +3575,9 @@ " └─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" ] }, { @@ -3464,6 +3592,9 @@ " └─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" ] }, { @@ -3478,6 +3609,9 @@ " └─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" ] }, { @@ -3492,6 +3626,9 @@ " └─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" ] }, { @@ -3507,6 +3644,27 @@ " └─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" ] } ] diff --git a/planner/optimize.go b/planner/optimize.go index e8536e8dbed34..68c0a12d871f8 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/cascades" "github.com/pingcap/tidb/planner/core" plannercore "github.com/pingcap/tidb/planner/core" @@ -500,13 +501,24 @@ func handleEvolveTasks(ctx context.Context, sctx sessionctx.Context, br *bindinf // useMaxTS returns true when meets following conditions: // 1. ctx is auto commit tagged. // 2. plan is point get by pk. +// 3. not a cache table. func useMaxTS(ctx sessionctx.Context, p plannercore.Plan) bool { if !plannercore.IsAutoCommitTxn(ctx) { return false } - v, ok := p.(*plannercore.PointGetPlan) - return ok && (v.IndexInfo == nil || (v.IndexInfo.Primary && v.TblInfo.IsCommonHandle)) + if !ok { + return false + } + noSecondRead := v.IndexInfo == nil || (v.IndexInfo.Primary && v.TblInfo.IsCommonHandle) + if !noSecondRead { + return false + } + + if v.TblInfo != nil && (v.TblInfo.TableCacheStatusType != model.TableCacheStatusDisable) { + return false + } + return true } // OptimizeExecStmt to optimize prepare statement protocol "execute" statement diff --git a/planner/util/path.go b/planner/util/path.go index 8c4abd6a94ab5..694ea781959aa 100644 --- a/planner/util/path.go +++ b/planner/util/path.go @@ -116,7 +116,7 @@ func isColEqCorColOrConstant(ctx sessionctx.Context, filter expression.Expressio if !ok || f.FuncName.L != ast.EQ { return false } - _, collation := f.CharsetAndCollation(ctx) + _, collation := f.CharsetAndCollation() if c, ok := f.GetArgs()[0].(*expression.Column); ok { if c.RetType.EvalType() == types.ETString && !collate.CompatibleCollate(collation, c.RetType.Collate) { return false diff --git a/server/conn_test.go b/server/conn_test.go index 65ae656773a56..06b4b3a38c926 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -544,7 +544,7 @@ func TestGetSessionVarsWaitTimeout(t *testing.T) { }, ctx: tc, } - require.Equal(t, uint64(0), cc.getSessionVarsWaitTimeout(context.Background())) + require.Equal(t, uint64(variable.DefWaitTimeout), cc.getSessionVarsWaitTimeout(context.Background())) } func mapIdentical(m1, m2 map[string]string) bool { diff --git a/server/server.go b/server/server.go index 36b564f44d410..e656569a14a77 100644 --- a/server/server.go +++ b/server/server.go @@ -618,6 +618,9 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { defer s.rwlock.RUnlock() rs := make(map[uint64]*util.ProcessInfo, len(s.clients)) for _, client := range s.clients { + if atomic.LoadInt32(&client.status) == connStatusWaitShutdown { + continue + } if pi := client.ctx.ShowProcess(); pi != nil { rs[pi.ID] = pi } diff --git a/session/session.go b/session/session.go index 12502a723b579..83c936ccf30b2 100644 --- a/session/session.go +++ b/session/session.go @@ -161,9 +161,7 @@ type Session interface { ClearDiskFullOpt() } -var ( - _ Session = (*session)(nil) -) +var _ Session = (*session)(nil) type stmtRecord struct { st sqlexec.Statement @@ -2083,7 +2081,7 @@ func (s *session) NewTxn(ctx context.Context) error { if err := s.checkBeforeNewTxn(ctx); err != nil { return err } - txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) + txn, err := s.store.Begin(tikv.WithTxnScope(s.sessionVars.CheckAndGetTxnScope())) if err != nil { return err } @@ -2128,7 +2126,7 @@ func (s *session) NewStaleTxnWithStartTS(ctx context.Context, startTS uint64) er return err } txnScope := config.GetTxnScopeFromConfig() - txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetStartTS(startTS)) + txn, err := s.store.Begin(tikv.WithTxnScope(txnScope), tikv.WithStartTS(startTS)) if err != nil { return err } @@ -2424,9 +2422,7 @@ func loadDefOOMAction(se *session) (string, error) { return defOOMAction, nil } -var ( - errResultIsEmpty = dbterror.ClassExecutor.NewStd(errno.ErrResultIsEmpty) -) +var errResultIsEmpty = dbterror.ClassExecutor.NewStd(errno.ErrResultIsEmpty) // BootstrapSession runs the first time when the TiDB server start. func BootstrapSession(store kv.Storage) (*domain.Domain, error) { @@ -2687,7 +2683,6 @@ func getStoreBootstrapVersion(store kv.Storage) int64 { ver, err = t.GetBootstrapVersion() return err }) - if err != nil { logutil.BgLogger().Fatal("check bootstrapped failed", zap.Error(err)) @@ -2826,7 +2821,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { } // no need to get txn from txnFutureCh since txn should init with startTs - txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTS(startTS)) + txn, err := s.store.Begin(tikv.WithTxnScope(s.GetSessionVars().CheckAndGetTxnScope()), tikv.WithStartTS(startTS)) if err != nil { return err } diff --git a/session/session_test.go b/session/session_test.go index 4811eaac34038..ceca24f509680 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -5718,26 +5718,28 @@ func (s *testSessionSuite) TestSetPDClientDynmaicOption(c *C) { var err error tk := testkit.NewTestKit(c, s.store) tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 0.5;") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0.5")) tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 1;") tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("1")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 1.5;") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("1.5")) tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 10;") tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("10")) err = tk.ExecToErr("set tidb_tso_client_batch_max_wait_time = 0;") c.Assert(err, NotNil) - if *withTiKV { - err = tk.ExecToErr("set global tidb_tso_client_batch_max_wait_time = -1;") - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, ".*invalid max TSO batch wait interval.*") - err = tk.ExecToErr("set global tidb_tso_client_batch_max_wait_time = 11;") - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, ".*invalid max TSO batch wait interval.*") - } else { - // Because the PD client in the unit test may be nil, so we only check the warning here. - tk.MustExec("set global tidb_tso_client_batch_max_wait_time = -1;") - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '-1'")) - tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 11;") - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '11'")) - } + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = -1;") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '-1'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = -0.1;") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '-0.1'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 10.1;") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '10.1'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("10")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 11;") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '11'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("10")) tk.MustQuery("select @@tidb_enable_tso_follower_proxy;").Check(testkit.Rows("0")) tk.MustExec("set global tidb_enable_tso_follower_proxy = on;") diff --git a/session/txn.go b/session/txn.go index 59472807a4c83..2ed83e89c2fde 100644 --- a/session/txn.go +++ b/session/txn.go @@ -489,14 +489,14 @@ type txnFuture struct { func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() if err == nil { - return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope).SetStartTS(startTS)) + return tf.store.Begin(tikv.WithTxnScope(tf.txnScope), tikv.WithStartTS(startTS)) } else if config.GetGlobalConfig().Store == "unistore" { return nil, err } logutil.BgLogger().Warn("wait tso failed", zap.Error(err)) // It would retry get timestamp. - return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope)) + return tf.store.Begin(tikv.WithTxnScope(tf.txnScope)) } func (s *session) getTxnFuture(ctx context.Context) *txnFuture { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 72bf0a5c27ed1..03c488c883a0c 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" + "github.com/pingcap/tidb/util/tracing" "github.com/tikv/client-go/v2/util" atomic2 "go.uber.org/atomic" "go.uber.org/zap" @@ -175,11 +176,9 @@ type StatementContext struct { // Map to store all CTE storages of current SQL. // Will clean up at the end of the execution. CTEStorageMap interface{} - // cachedTables is used to store cache table id and a pointer to cache data when it satisfies the cache read condition - cachedTables []struct { - id int64 - memBuffer interface{} // is a point to cache.MemBuffer. in order to avoid import cycle - } + + // If the statement read from table cache, this flag is set. + ReadFromTableCache bool // cache is used to reduce object allocation. cache struct { @@ -193,6 +192,11 @@ type StatementContext struct { OptimInfo map[int]string // InVerboseExplain indicates the statement is "explain format='verbose' ...". InVerboseExplain bool + + // EnableOptimizeTrace indicates whether the statement is enable optimize trace + EnableOptimizeTrace bool + // LogicalOptimizeTrace indicates the trace for optimize + LogicalOptimizeTrace *tracing.LogicalOptimizeTracer } // StmtHints are SessionVars related sql hints. @@ -331,35 +335,6 @@ func (sc *StatementContext) SetPlanHint(hint string) { sc.planHint = hint } -// StoreCacheTable stores the read condition and a point to cache data of the given key. -func (sc *StatementContext) StoreCacheTable(tblID int64, buffer interface{}) { - for _, data := range sc.cachedTables { - if data.id == tblID { - data.memBuffer = buffer - } - return - } - sc.cachedTables = append(sc.cachedTables, struct { - id int64 - memBuffer interface{} - }{id: tblID, memBuffer: buffer}) -} - -// GetCacheTable gets the read condition and a point to cache data of the given key if it exists -func (sc *StatementContext) GetCacheTable(tblID int64) (bool, interface{}) { - for _, data := range sc.cachedTables { - if data.id == tblID { - return true, data.memBuffer - } - } - return false, nil -} - -// CacheTableUsed is used by test to check whether the last query use table cache. -func (sc *StatementContext) CacheTableUsed() bool { - return len(sc.cachedTables) > 0 -} - // TableEntry presents table in db. type TableEntry struct { DB string diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 913f09b93e486..c395d98c85356 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" @@ -1258,6 +1259,7 @@ func NewSessionVars() *SessionVars { if !EnableLocalTxn.Load() { vars.TxnScope = kv.NewGlobalTxnScopeVar() } + vars.systems[CharacterSetConnection], vars.systems[CollationConnection] = charset.GetDefaultCharsetAndCollate() return vars } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b320e3a60fd66..4d0a77a5cb511 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -807,12 +807,14 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: InitConnect, Value: ""}, /* TiDB specific variables */ - {Scope: ScopeGlobal, Name: TiDBTSOClientBatchMaxWaitTime, Value: strconv.Itoa(DefTiDBTSOClientBatchMaxWaitTime), Type: TypeInt, MinValue: 0, MaxValue: 10, GetGlobal: func(sv *SessionVars) (string, error) { - return strconv.Itoa(int(MaxTSOBatchWaitInterval.Load())), nil - }, SetGlobal: func(s *SessionVars, val string) error { - MaxTSOBatchWaitInterval.Store(tidbOptInt64(val, DefTiDBTSOClientBatchMaxWaitTime)) - return nil - }}, + {Scope: ScopeGlobal, Name: TiDBTSOClientBatchMaxWaitTime, Value: strconv.FormatFloat(DefTiDBTSOClientBatchMaxWaitTime, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 10, + GetGlobal: func(sv *SessionVars) (string, error) { + return strconv.FormatFloat(MaxTSOBatchWaitInterval.Load(), 'f', -1, 64), nil + }, + SetGlobal: func(s *SessionVars, val string) error { + MaxTSOBatchWaitInterval.Store(tidbOptFloat64(val, DefTiDBTSOClientBatchMaxWaitTime)) + return nil + }}, {Scope: ScopeGlobal, Name: TiDBEnableTSOFollowerProxy, Value: BoolToOnOff(DefTiDBEnableTSOFollowerProxy), Type: TypeBool, GetGlobal: func(sv *SessionVars) (string, error) { return BoolToOnOff(EnableTSOFollowerProxy.Load()), nil }, SetGlobal: func(s *SessionVars, val string) error { diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index ea461d2864ed2..8ae408eee6d67 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -807,3 +807,13 @@ func TestDDLWorkers(t *testing.T) { require.NoError(t, err) require.Equal(t, val, "100") // unchanged } + +func TestDefaultCharsetAndCollation(t *testing.T) { + vars := NewSessionVars() + val, err := GetSessionOrGlobalSystemVar(vars, CharacterSetConnection) + require.NoError(t, err) + require.Equal(t, val, mysql.DefaultCharset) + val, err = GetSessionOrGlobalSystemVar(vars, CollationConnection) + require.NoError(t, err) + require.Equal(t, val, mysql.DefaultCollationName) +} diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 0fca472a4f336..ef7c001841d13 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -672,7 +672,7 @@ const ( DefMaxChunkSize = 1024 DefDMLBatchSize = 0 DefMaxPreparedStmtCount = -1 - DefWaitTimeout = 0 + DefWaitTimeout = 28800 DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB. DefTiDBMemQuotaHashJoin = 32 << 30 // 32GB. DefTiDBMemQuotaMergeJoin = 32 << 30 // 32GB. @@ -764,7 +764,7 @@ const ( DefTiDBTopSQLReportIntervalSeconds = 60 DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. DefTiDBEnableLocalTxn = false - DefTiDBTSOClientBatchMaxWaitTime = 0 // 0ms + DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms DefTiDBEnableTSOFollowerProxy = false DefTiDBEnableOrderedResultMode = false DefTiDBEnablePseudoForOutdatedStats = true @@ -802,7 +802,7 @@ var ( ReportIntervalSeconds: atomic.NewInt64(DefTiDBTopSQLReportIntervalSeconds), } EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) - MaxTSOBatchWaitInterval = atomic.NewInt64(DefTiDBTSOClientBatchMaxWaitTime) + MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) ) diff --git a/statistics/handle/ddl_test.go b/statistics/handle/ddl_serial_test.go similarity index 99% rename from statistics/handle/ddl_test.go rename to statistics/handle/ddl_serial_test.go index b7bc224f7f5ee..76121694338df 100644 --- a/statistics/handle/ddl_test.go +++ b/statistics/handle/ddl_serial_test.go @@ -26,7 +26,6 @@ import ( ) func TestDDLAfterLoad(t *testing.T) { - t.Parallel() testKit, do, clean := createTestKitAndDom(t) defer clean() testKit.MustExec("use test") @@ -60,7 +59,6 @@ func TestDDLAfterLoad(t *testing.T) { } func TestDDLTable(t *testing.T) { - t.Parallel() testKit, do, clean := createTestKitAndDom(t) defer clean() testKit.MustExec("use test") @@ -100,7 +98,6 @@ func TestDDLTable(t *testing.T) { } func TestDDLHistogram(t *testing.T) { - t.Parallel() testKit, do, clean := createTestKitAndDom(t) defer clean() h := do.StatsHandle() @@ -188,7 +185,6 @@ func TestDDLHistogram(t *testing.T) { } func TestDDLPartition(t *testing.T) { - t.Parallel() testKit, do, clean := createTestKitAndDom(t) defer clean() testkit.WithPruneMode(testKit, variable.Static, func() { diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_serial_test.go similarity index 57% rename from statistics/handle/dump_test.go rename to statistics/handle/dump_serial_test.go index 2ce75f722d2d0..a70bc43fb750f 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_serial_test.go @@ -18,17 +18,64 @@ import ( "encoding/json" "fmt" "sync" + "testing" - . "github.com/pingcap/check" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -func (s *testStatsSuite) TestConversion(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func requireTableEqual(t *testing.T, a *statistics.Table, b *statistics.Table) { + require.Equal(t, b.Count, a.Count) + require.Equal(t, b.ModifyCount, a.ModifyCount) + require.Equal(t, len(b.Columns), len(a.Columns)) + for i := range a.Columns { + 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 { + require.Nil(t, b.Columns[i].CMSketch) + } else { + 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. + require.Truef(t, a.Columns[i].TopN.Equal(b.Columns[i].TopN), "%v, %v", a.Columns[i].TopN, b.Columns[i].TopN) + } + require.Equal(t, len(b.Indices), len(a.Indices)) + for i := range a.Indices { + require.True(t, statistics.HistogramEqual(&a.Indices[i].Histogram, &b.Indices[i].Histogram, false)) + if a.Indices[i].CMSketch == nil { + require.Nil(t, b.Indices[i].CMSketch) + } else { + require.True(t, a.Indices[i].CMSketch.Equal(b.Indices[i].CMSketch)) + } + require.True(t, a.Indices[i].TopN.Equal(b.Indices[i].TopN)) + } + require.True(t, isSameExtendedStats(a.ExtendedStats, b.ExtendedStats)) +} + +func cleanStats(tk *testkit.TestKit, do *domain.Domain) { + 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") + do.StatsHandle().Clear() +} + +func TestConversion(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() tk.MustExec("use test") tk.MustExec("create table t (a int, b int)") @@ -36,50 +83,50 @@ func (s *testStatsSuite) TestConversion(c *C) { tk.MustExec("insert into t(a,b) values (3, 1),(2, 1),(1, 10)") tk.MustExec("analyze table t") tk.MustExec("insert into t(a,b) values (1, 1),(3, 1),(5, 10)") - is := s.do.InfoSchema() - h := s.do.StatsHandle() - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + is := dom.InfoSchema() + h := dom.StatsHandle() + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Nil(t, h.Update(is)) tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil) - c.Assert(err, IsNil) + require.NoError(t, err) loadTbl, err := handle.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, jsonTbl) - c.Assert(err, IsNil) + require.NoError(t, err) tbl := h.GetTableStats(tableInfo.Meta()) - assertTableEqual(c, loadTbl, tbl) + requireTableEqual(t, loadTbl, tbl) - cleanEnv(c, s.store, s.do) + cleanStats(tk, dom) wg := sync.WaitGroup{} wg.Add(1) go func() { - c.Assert(h.Update(is), IsNil) + require.Nil(t, h.Update(is)) wg.Done() }() err = h.LoadStatsFromJSON(is, jsonTbl) wg.Wait() - c.Assert(err, IsNil) + require.NoError(t, err) loadTblInStorage := h.GetTableStats(tableInfo.Meta()) - assertTableEqual(c, loadTblInStorage, tbl) + requireTableEqual(t, loadTblInStorage, tbl) } -func (s *testStatsSuite) getStatsJSON(c *C, db, tableName string) *handle.JSONTable { - is := s.do.InfoSchema() - h := s.do.StatsHandle() - c.Assert(h.Update(is), IsNil) +func getStatsJSON(t *testing.T, dom *domain.Domain, db, tableName string) *handle.JSONTable { + is := dom.InfoSchema() + h := dom.StatsHandle() + require.Nil(t, h.Update(is)) table, err := is.TableByName(model.NewCIStr(db), model.NewCIStr(tableName)) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := table.Meta() jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil) - c.Assert(err, IsNil) + require.NoError(t, err) return jsonTbl } -func (s *testStatsSuite) TestDumpGlobalStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDumpGlobalStats(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() tk.MustExec("use test") tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("set @@tidb_partition_prune_mode = 'static'") @@ -89,23 +136,23 @@ func (s *testStatsSuite) TestDumpGlobalStats(c *C) { tk.MustExec("analyze table t") // global-stats is not existed - stats := s.getStatsJSON(c, "test", "t") - c.Assert(stats.Partitions["p0"], NotNil) - c.Assert(stats.Partitions["p1"], NotNil) - c.Assert(stats.Partitions["global"], IsNil) + stats := getStatsJSON(t, dom, "test", "t") + require.NotNil(t, stats.Partitions["p0"]) + require.NotNil(t, stats.Partitions["p1"]) + require.Nil(t, stats.Partitions["global"]) // global-stats is existed tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("analyze table t") - stats = s.getStatsJSON(c, "test", "t") - c.Assert(stats.Partitions["p0"], NotNil) - c.Assert(stats.Partitions["p1"], NotNil) - c.Assert(stats.Partitions["global"], NotNil) + stats = getStatsJSON(t, dom, "test", "t") + require.NotNil(t, stats.Partitions["p0"]) + require.NotNil(t, stats.Partitions["p1"]) + require.NotNil(t, stats.Partitions["global"]) } -func (s *testStatsSuite) TestLoadGlobalStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestLoadGlobalStats(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() tk.MustExec("use test") tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -113,25 +160,25 @@ func (s *testStatsSuite) TestLoadGlobalStats(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)") tk.MustExec("analyze table t") - globalStats := s.getStatsJSON(c, "test", "t") + globalStats := getStatsJSON(t, dom, "test", "t") // remove all statistics tk.MustExec("delete from mysql.stats_meta") tk.MustExec("delete from mysql.stats_histograms") tk.MustExec("delete from mysql.stats_buckets") - s.do.StatsHandle().Clear() - clearedStats := s.getStatsJSON(c, "test", "t") - c.Assert(len(clearedStats.Partitions), Equals, 0) + dom.StatsHandle().Clear() + clearedStats := getStatsJSON(t, dom, "test", "t") + require.Equal(t, 0, len(clearedStats.Partitions)) // load global-stats back - c.Assert(s.do.StatsHandle().LoadStatsFromJSON(s.do.InfoSchema(), globalStats), IsNil) - loadedStats := s.getStatsJSON(c, "test", "t") - c.Assert(len(loadedStats.Partitions), Equals, 3) // p0, p1, global + require.Nil(t, dom.StatsHandle().LoadStatsFromJSON(dom.InfoSchema(), globalStats)) + loadedStats := getStatsJSON(t, dom, "test", "t") + require.Equal(t, 3, len(loadedStats.Partitions)) // p0, p1, global } -func (s *testStatsSuite) TestDumpPartitions(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDumpPartitions(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() 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)) @@ -146,15 +193,15 @@ PARTITION BY RANGE ( a ) ( tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d)`, i, i)) } tk.MustExec("analyze table t") - is := s.do.InfoSchema() - h := s.do.StatsHandle() - c.Assert(h.Update(is), IsNil) + is := dom.InfoSchema() + h := dom.StatsHandle() + require.Nil(t, h.Update(is)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := table.Meta() jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil) - c.Assert(err, IsNil) + require.NoError(t, err) pi := tableInfo.GetPartitionInfo() originTables := make([]*statistics.Table, 0, len(pi.Definitions)) for _, def := range pi.Definitions { @@ -166,47 +213,47 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("delete from mysql.stats_buckets") h.Clear() - err = h.LoadStatsFromJSON(s.do.InfoSchema(), jsonTbl) - c.Assert(err, IsNil) + err = h.LoadStatsFromJSON(dom.InfoSchema(), jsonTbl) + require.NoError(t, err) for i, def := range pi.Definitions { - t := h.GetPartitionStats(tableInfo, def.ID) - assertTableEqual(c, originTables[i], t) + tt := h.GetPartitionStats(tableInfo, def.ID) + requireTableEqual(t, originTables[i], tt) } } -func (s *testStatsSuite) TestDumpAlteredTable(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDumpAlteredTable(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() tk.MustExec("use test") tk.MustExec("drop table if exists t") - h := s.do.StatsHandle() + h := dom.StatsHandle() oriLease := h.Lease() h.SetLease(1) defer func() { h.SetLease(oriLease) }() tk.MustExec("create table t(a int, b int)") tk.MustExec("analyze table t") tk.MustExec("alter table t drop column a") - table, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) _, err = h.DumpStatsToJSON("test", table.Meta(), nil) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testStatsSuite) TestDumpCMSketchWithTopN(c *C) { +func TestDumpCMSketchWithTopN(t *testing.T) { // Just test if we can store and recover the Top N elements stored in database. - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) + testKit, dom, clean := createTestKitAndDom(t) + defer clean() testKit.MustExec("use test") testKit.MustExec("create table t(a int)") testKit.MustExec("insert into t values (1),(3),(4),(2),(5)") 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() - c.Assert(h.Update(is), IsNil) + h := dom.StatsHandle() + require.Nil(t, h.Update(is)) // Insert 30 fake data fakeData := make([][]byte, 0, 30) @@ -217,80 +264,80 @@ func (s *testStatsSuite) TestDumpCMSketchWithTopN(c *C) { stat := h.GetTableStats(tableInfo) err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, nil, statistics.Version2, 1, false, false) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.Nil(t, h.Update(is)) stat = h.GetTableStats(tableInfo) cmsFromStore := stat.Columns[tableInfo.Columns[0].ID].CMSketch - c.Assert(cmsFromStore, NotNil) - c.Check(cms.Equal(cmsFromStore), IsTrue) + require.NotNil(t, cmsFromStore) + require.True(t, cms.Equal(cmsFromStore)) jsonTable, err := h.DumpStatsToJSON("test", tableInfo, nil) - c.Check(err, IsNil) + require.NoError(t, err) err = h.LoadStatsFromJSON(is, jsonTable) - c.Check(err, IsNil) + require.NoError(t, err) stat = h.GetTableStats(tableInfo) cmsFromJSON := stat.Columns[tableInfo.Columns[0].ID].CMSketch.Copy() - c.Check(cms.Equal(cmsFromJSON), IsTrue) + require.True(t, cms.Equal(cmsFromJSON)) } -func (s *testStatsSuite) TestDumpPseudoColumns(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestDumpPseudoColumns(t *testing.T) { + testKit, dom, clean := createTestKitAndDom(t) + defer clean() testKit.MustExec("use test") testKit.MustExec("create table t(a int, b int, index idx(a))") // Force adding an pseudo tables in stats cache. testKit.MustQuery("select * from t") testKit.MustExec("analyze table t index idx") - is := s.do.InfoSchema() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - h := s.do.StatsHandle() + require.NoError(t, err) + h := dom.StatsHandle() _, err = h.DumpStatsToJSON("test", tbl.Meta(), nil) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testStatsSuite) TestDumpExtendedStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDumpExtendedStats(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() tk.MustExec("set session tidb_enable_extended_stats = on") 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(1,5),(2,4),(3,3),(4,2),(5,1)") - h := s.do.StatsHandle() - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + h := dom.StatsHandle() + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustExec("analyze table t") - is := s.do.InfoSchema() + is := dom.InfoSchema() tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tbl := h.GetTableStats(tableInfo.Meta()) jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil) - c.Assert(err, IsNil) + require.NoError(t, err) loadTbl, err := handle.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, jsonTbl) - c.Assert(err, IsNil) - assertTableEqual(c, loadTbl, tbl) + require.NoError(t, err) + requireTableEqual(t, loadTbl, tbl) - cleanEnv(c, s.store, s.do) + cleanStats(tk, dom) wg := sync.WaitGroup{} wg.Add(1) go func() { - c.Assert(h.Update(is), IsNil) + require.Nil(t, h.Update(is)) wg.Done() }() err = h.LoadStatsFromJSON(is, jsonTbl) wg.Wait() - c.Assert(err, IsNil) + require.NoError(t, err) loadTblInStorage := h.GetTableStats(tableInfo.Meta()) - assertTableEqual(c, loadTblInStorage, tbl) + requireTableEqual(t, loadTblInStorage, tbl) } -func (s *testStatsSuite) TestDumpVer2Stats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDumpVer2Stats(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -302,39 +349,39 @@ func (s *testStatsSuite) TestDumpVer2Stats(c *C) { tk.MustExec("alter table t add index single(a)") tk.MustExec("alter table t add index multi(a, b)") tk.MustExec("analyze table t with 2 topn") - h := s.do.StatsHandle() - is := s.do.InfoSchema() + h := dom.StatsHandle() + is := dom.InfoSchema() tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) storageTbl, err := h.TableStatsFromStorage(tableInfo.Meta(), tableInfo.Meta().ID, false, 0) - c.Assert(err, IsNil) + require.NoError(t, err) dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil) - c.Assert(err, IsNil) + require.NoError(t, err) jsonBytes, err := json.MarshalIndent(dumpJSONTable, "", " ") - c.Assert(err, IsNil) + require.NoError(t, err) loadJSONTable := &handle.JSONTable{} err = json.Unmarshal(jsonBytes, loadJSONTable) - c.Assert(err, IsNil) + require.NoError(t, err) loadTbl, err := handle.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, loadJSONTable) - c.Assert(err, IsNil) + require.NoError(t, err) // assert that a statistics.Table from storage dumped into JSON text and then unmarshalled into a statistics.Table keeps unchanged - assertTableEqual(c, loadTbl, storageTbl) + requireTableEqual(t, loadTbl, storageTbl) // assert that this statistics.Table is the same as the one in stats cache statsCacheTbl := h.GetTableStats(tableInfo.Meta()) - assertTableEqual(c, loadTbl, statsCacheTbl) + requireTableEqual(t, loadTbl, statsCacheTbl) err = h.LoadStatsFromJSON(is, loadJSONTable) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.Nil(t, h.Update(is)) statsCacheTbl = h.GetTableStats(tableInfo.Meta()) // assert that after the JSONTable above loaded into storage then updated into the stats cache, // the statistics.Table in the stats cache is the same as the unmarshalled statistics.Table - assertTableEqual(c, statsCacheTbl, loadTbl) + requireTableEqual(t, statsCacheTbl, loadTbl) } diff --git a/statistics/handle/gc_test.go b/statistics/handle/gc_serial_test.go similarity index 99% rename from statistics/handle/gc_test.go rename to statistics/handle/gc_serial_test.go index eb1c3504e4c05..b426613952c4d 100644 --- a/statistics/handle/gc_test.go +++ b/statistics/handle/gc_serial_test.go @@ -37,7 +37,6 @@ func createTestKitAndDom(t *testing.T) (*testkit.TestKit, *domain.Domain, func() } func TestGCStats(t *testing.T) { - t.Parallel() testKit, dom, clean := createTestKitAndDom(t) defer clean() testKit.MustExec("set @@tidb_analyze_version = 1") @@ -71,7 +70,6 @@ func TestGCStats(t *testing.T) { } func TestGCPartition(t *testing.T) { - t.Parallel() testKit, dom, clean := createTestKitAndDom(t) defer clean() testKit.MustExec("set @@tidb_analyze_version = 1") @@ -110,7 +108,6 @@ func TestGCPartition(t *testing.T) { } func TestGCExtendedStats(t *testing.T) { - t.Parallel() testKit, dom, clean := createTestKitAndDom(t) defer clean() testKit.MustExec("set session tidb_enable_extended_stats = on") @@ -155,7 +152,6 @@ func TestGCExtendedStats(t *testing.T) { } func TestGCColumnStatsUsage(t *testing.T) { - t.Parallel() testKit, dom, clean := createTestKitAndDom(t) defer clean() testKit.MustExec("use test") diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index d8c7512cb1763..8c5f1d08a4013 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -49,7 +49,7 @@ func TestT(t *testing.T) { TestingT(t) } -// TODO replace cleanEnv with createTestKitAndDom in gc_test.go when migrate this file +// 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") diff --git a/statistics/selectivity_serial_test.go b/statistics/selectivity_serial_test.go index 43435417f7229..7fdbf09c757dc 100644 --- a/statistics/selectivity_serial_test.go +++ b/statistics/selectivity_serial_test.go @@ -16,14 +16,20 @@ package statistics_test import ( "context" + "fmt" + "math" "os" "runtime/pprof" "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" plannercore "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/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/testkit" @@ -101,3 +107,681 @@ func BenchmarkSelectivity(b *testing.B) { }) pprof.StopCPUProfile() } + +func TestOutOfRangeEstimation(t *testing.T) { + domain.RunAutoAnalyze = false + 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 unsigned)") + for i := 0; i < 3000; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%v)", i/5+300)) // [300, 900) + } + testKit.MustExec("analyze table t with 2000 samples") + + h := dom.StatsHandle() + table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + statsTbl := h.GetTableStats(table.Meta()) + sc := &stmtctx.StatementContext{} + col := statsTbl.Columns[table.Meta().Columns[0].ID] + count, err := col.GetColumnRowCount(sc, getRange(900, 900), statsTbl.Count, false) + require.NoError(t, err) + // Because the ANALYZE collect data by random sampling, so the result is not an accurate value. + // so we use a range here. + require.Truef(t, count < 5.5, "expected: around 5.0, got: %v", count) + require.Truef(t, count > 4.5, "expected: around 5.0, got: %v", count) + + var input []struct { + Start int64 + End int64 + } + var output []struct { + Start int64 + End int64 + Count float64 + } + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + increasedTblRowCount := int64(float64(statsTbl.Count) * 1.5) + for i, ran := range input { + count, err = col.GetColumnRowCount(sc, getRange(ran.Start, ran.End), increasedTblRowCount, false) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].Start = ran.Start + output[i].End = ran.End + output[i].Count = count + }) + require.Truef(t, count < output[i].Count*1.2, "for [%v, %v], needed: around %v, got: %v", ran.Start, ran.End, output[i].Count, count) + require.Truef(t, count > output[i].Count*0.8, "for [%v, %v], needed: around %v, got: %v", ran.Start, ran.End, output[i].Count, count) + } +} + +func TestEstimationForUnknownValues(t *testing.T) { + domain.RunAutoAnalyze = false + 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, key idx(a, b))") + testKit.MustExec("set @@tidb_analyze_version=1") + testKit.MustExec("analyze table t") + for i := 0; i < 10; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) + } + h := dom.StatsHandle() + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + testKit.MustExec("analyze table t") + for i := 0; i < 10; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i+10, i+10)) + } + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Nil(t, h.Update(dom.InfoSchema())) + table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + statsTbl := h.GetTableStats(table.Meta()) + + sc := &stmtctx.StatementContext{} + colID := table.Meta().Columns[0].ID + count, err := statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(30, 30)) + require.NoError(t, err) + require.Equal(t, 0.2, count) + + count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(9, 30)) + require.NoError(t, err) + require.Equal(t, 7.2, count) + + count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(9, math.MaxInt64)) + require.NoError(t, err) + require.Equal(t, 7.2, count) + + idxID := table.Meta().Indices[0].ID + count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(30, 30)) + require.NoError(t, err) + require.Equal(t, 0.1, count) + + count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(9, 30)) + require.NoError(t, err) + require.Equal(t, 7.0, count) + + testKit.MustExec("truncate table t") + testKit.MustExec("insert into t values (null, null)") + testKit.MustExec("analyze table t") + table, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + statsTbl = h.GetTableStats(table.Meta()) + + colID = table.Meta().Columns[0].ID + count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(1, 30)) + require.NoError(t, err) + require.Equal(t, 0.0, count) + + testKit.MustExec("drop table t") + testKit.MustExec("create table t(a int, b int, index idx(b))") + testKit.MustExec("insert into t values (1,1)") + testKit.MustExec("analyze table t") + table, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + statsTbl = h.GetTableStats(table.Meta()) + + colID = table.Meta().Columns[0].ID + count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(2, 2)) + require.NoError(t, err) + require.Equal(t, 0.0, count) + + idxID = table.Meta().Indices[0].ID + count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(2, 2)) + require.NoError(t, err) + require.Equal(t, 0.0, count) +} + +func TestEstimationUniqueKeyEqualConds(t *testing.T) { + domain.RunAutoAnalyze = false + 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, c int, unique key(b))") + testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7)") + testKit.MustExec("analyze table t with 4 cmsketch width, 1 cmsketch depth;") + table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + statsTbl := dom.StatsHandle().GetTableStats(table.Meta()) + + sc := &stmtctx.StatementContext{} + idxID := table.Meta().Indices[0].ID + count, err := statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(7, 7)) + require.NoError(t, err) + require.Equal(t, 1.0, count) + + count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(6, 6)) + require.NoError(t, err) + require.Equal(t, 1.0, count) + + colID := table.Meta().Columns[0].ID + count, err = statsTbl.GetRowCountByIntColumnRanges(sc, colID, getRange(7, 7)) + require.NoError(t, err) + require.Equal(t, 1.0, count) + + count, err = statsTbl.GetRowCountByIntColumnRanges(sc, colID, getRange(6, 6)) + require.NoError(t, err) + require.Equal(t, 1.0, count) +} + +func TestPrimaryKeySelectivity(t *testing.T) { + domain.RunAutoAnalyze = false + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + testKit.MustExec("create table t(a char(10) primary key, b int)") + var input, output [][]string + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + for i, ts := range input { + for j, tt := range ts { + if j != len(ts)-1 { + testKit.MustExec(tt) + } + testdata.OnRecord(func() { + if j == len(ts)-1 { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + } + }) + if j == len(ts)-1 { + testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) + } + } + } +} + +func TestStatsVer2(t *testing.T) { + domain.RunAutoAnalyze = false + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("set tidb_analyze_version=2") + + testKit.MustExec("drop table if exists tint") + testKit.MustExec("create table tint(a int, b int, c int, index singular(a), index multi(b, c))") + testKit.MustExec("insert into tint 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)") + testKit.MustExec("analyze table tint with 2 topn, 3 buckets") + + testKit.MustExec("drop table if exists tdouble") + testKit.MustExec("create table tdouble(a double, b double, c double, index singular(a), index multi(b, c))") + testKit.MustExec("insert into tdouble 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)") + testKit.MustExec("analyze table tdouble with 2 topn, 3 buckets") + + testKit.MustExec("drop table if exists tdecimal") + testKit.MustExec("create table tdecimal(a decimal(40, 20), b decimal(40, 20), c decimal(40, 20), index singular(a), index multi(b, c))") + testKit.MustExec("insert into tdecimal 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)") + testKit.MustExec("analyze table tdecimal with 2 topn, 3 buckets") + + testKit.MustExec("drop table if exists tstring") + testKit.MustExec("create table tstring(a varchar(64), b varchar(64), c varchar(64), index singular(a), index multi(b, c))") + testKit.MustExec("insert into tstring 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')") + testKit.MustExec("analyze table tstring with 2 topn, 3 buckets") + + testKit.MustExec("drop table if exists tdatetime") + testKit.MustExec("create table tdatetime(a datetime, b datetime, c datetime, index singular(a), index multi(b, c))") + testKit.MustExec("insert into tdatetime values ('2001-01-01', '2001-01-01', '2001-01-01'), ('2001-01-02', '2001-01-02', '2001-01-02'), ('2001-01-03', '2001-01-03', '2001-01-03'), ('2001-01-04', '2001-01-04', '2001-01-04')") + testKit.MustExec("analyze table tdatetime with 2 topn, 3 buckets") + + testKit.MustExec("drop table if exists tprefix") + testKit.MustExec("create table tprefix(a varchar(64), b varchar(64), index prefixa(a(2)))") + testKit.MustExec("insert into tprefix values ('111', '111'), ('222', '222'), ('333', '333'), ('444', '444'), ('555', '555'), ('666', '666')") + testKit.MustExec("analyze table tprefix with 2 topn, 3 buckets") + + // test with clustered index + testKit.MustExec("drop table if exists ct1") + testKit.MustExec("create table ct1 (a int, pk varchar(10), primary key(pk) clustered)") + testKit.MustExec("insert into ct1 values (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'), (7, '7'), (8, '8')") + testKit.MustExec("analyze table ct1 with 2 topn, 3 buckets") + + testKit.MustExec("drop table if exists ct2") + testKit.MustExec("create table ct2 (a int, b int, c int, primary key(a, b) clustered)") + testKit.MustExec("insert into ct2 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)") + testKit.MustExec("analyze table ct2 with 2 topn, 3 buckets") + + rows := testKit.MustQuery("select stats_ver from mysql.stats_histograms").Rows() + for _, r := range rows { + // ensure statsVer = 2 + require.Equal(t, "2", fmt.Sprintf("%v", r[0])) + } + + var ( + input []string + output [][]string + ) + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + for i := range input { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} + +func TestTopNOutOfHist(t *testing.T) { + domain.RunAutoAnalyze = false + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("set tidb_analyze_version=2") + + testKit.MustExec("drop table if exists topn_before_hist") + testKit.MustExec("create table topn_before_hist(a int, index idx(a))") + testKit.MustExec("insert into topn_before_hist values(1), (1), (1), (1), (3), (3), (4), (5), (6)") + testKit.MustExec("analyze table topn_before_hist with 2 topn, 3 buckets") + + testKit.MustExec("create table topn_after_hist(a int, index idx(a))") + testKit.MustExec("insert into topn_after_hist values(2), (2), (3), (4), (5), (7), (7), (7), (7)") + testKit.MustExec("analyze table topn_after_hist with 2 topn, 3 buckets") + + testKit.MustExec("create table topn_before_hist_no_index(a int)") + testKit.MustExec("insert into topn_before_hist_no_index values(1), (1), (1), (1), (3), (3), (4), (5), (6)") + testKit.MustExec("analyze table topn_before_hist_no_index with 2 topn, 3 buckets") + + testKit.MustExec("create table topn_after_hist_no_index(a int)") + testKit.MustExec("insert into topn_after_hist_no_index values(2), (2), (3), (4), (5), (7), (7), (7), (7)") + testKit.MustExec("analyze table topn_after_hist_no_index with 2 topn, 3 buckets") + + var ( + input []string + output [][]string + ) + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + for i := range input { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} + +func TestColumnIndexNullEstimation(t *testing.T) { + domain.RunAutoAnalyze = false + 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, c int, index idx_b(b), index idx_c_a(c, a))") + testKit.MustExec("insert into t values(1,null,1),(2,null,2),(3,3,3),(4,null,4),(null,null,null);") + h := dom.StatsHandle() + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + testKit.MustExec("analyze table t") + var ( + input []string + output [][]string + ) + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + for i := 0; i < 5; i++ { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } + // Make sure column stats has been loaded. + testKit.MustExec(`explain select * from t where a is null`) + require.Nil(t, h.LoadNeededHistograms()) + for i := 5; i < len(input); i++ { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} + +func TestUniqCompEqualEst(t *testing.T) { + domain.RunAutoAnalyze = false + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int, b int, primary key(a, b))") + testKit.MustExec("insert into t values(1,1),(1,2),(1,3),(1,4),(1,5),(1,6),(1,7),(1,8),(1,9),(1,10)") + h := dom.StatsHandle() + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + testKit.MustExec("analyze table t") + var ( + input []string + output [][]string + ) + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + for i := 0; i < 1; i++ { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} + +func TestSelectivity(t *testing.T) { + domain.RunAutoAnalyze = false + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + statsTbl, err := prepareSelectivity(testKit, dom) + require.NoError(t, err) + longExpr := "0 < a and a = 1 " + for i := 1; i < 64; i++ { + longExpr += fmt.Sprintf(" and a > %d ", i) + } + tests := []struct { + exprs string + selectivity float64 + selectivityAfterIncrease float64 + }{ + { + exprs: "a > 0 and a < 2", + selectivity: 0.01851851851, + selectivityAfterIncrease: 0.01851851851, + }, + { + exprs: "a >= 1 and a < 2", + selectivity: 0.01851851851, + selectivityAfterIncrease: 0.01851851851, + }, + { + exprs: "a >= 1 and b > 1 and a < 2", + selectivity: 0.01783264746, + selectivityAfterIncrease: 0.01851851852, + }, + { + exprs: "a >= 1 and c > 1 and a < 2", + selectivity: 0.00617283950, + selectivityAfterIncrease: 0.00617283950, + }, + { + exprs: "a >= 1 and c >= 1 and a < 2", + selectivity: 0.01234567901, + selectivityAfterIncrease: 0.01234567901, + }, + { + exprs: "d = 0 and e = 1", + selectivity: 0.11111111111, + selectivityAfterIncrease: 0.11111111111, + }, + { + exprs: "b > 1", + selectivity: 0.96296296296, + selectivityAfterIncrease: 1, + }, + { + exprs: "a > 1 and b < 2 and c > 3 and d < 4 and e > 5", + selectivity: 0, + selectivityAfterIncrease: 0, + }, + { + exprs: longExpr, + selectivity: 0.001, + selectivityAfterIncrease: 0.001, + }, + } + + ctx := context.Background() + for _, tt := range tests { + sql := "select * from t where " + tt.exprs + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, sql) + require.NoErrorf(t, err, "for %s", tt.exprs) + require.Len(t, stmts, 1) + + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoErrorf(t, err, "for expr %s", tt.exprs) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoErrorf(t, err, "for building plan, expr %s", err, tt.exprs) + + sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + ds := sel.Children()[0].(*plannercore.DataSource) + + histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) + + ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) + require.NoErrorf(t, err, "for %s", tt.exprs) + require.Truef(t, math.Abs(ratio-tt.selectivity) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio) + + histColl.Count *= 10 + ratio, _, err = histColl.Selectivity(sctx, sel.Conditions, nil) + require.NoErrorf(t, err, "for %s", tt.exprs) + require.Truef(t, math.Abs(ratio-tt.selectivityAfterIncrease) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivityAfterIncrease, ratio) + } +} + +// TestDiscreteDistribution tests the estimation for discrete data distribution. This is more common when the index +// consists several columns, and the first column has small NDV. +func TestDiscreteDistribution(t *testing.T) { + domain.RunAutoAnalyze = false + 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 char(10), b int, key idx(a, b))") + for i := 0; i < 499; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values ('cn', %d)", i)) + } + for i := 0; i < 10; i++ { + testKit.MustExec("insert into t values ('tw', 0)") + } + testKit.MustExec("analyze table t") + var ( + input []string + output [][]string + ) + + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + + for i, tt := range input { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + }) + testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) + } +} + +func TestSelectCombinedLowBound(t *testing.T) { + domain.RunAutoAnalyze = false + 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(id int auto_increment, kid int, pid int, primary key(id), key(kid, pid))") + testKit.MustExec("insert into t (kid, pid) values (1,2), (1,3), (1,4),(1, 11), (1, 12), (1, 13), (1, 14), (2, 2), (2, 3), (2, 4)") + testKit.MustExec("analyze table t") + var ( + input []string + output [][]string + ) + + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + + for i, tt := range input { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + }) + testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) + } +} + +// TestDNFCondSelectivity tests selectivity calculation with DNF conditions covered by using independence assumption. +func TestDNFCondSelectivity(t *testing.T) { + domain.RunAutoAnalyze = false + 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, c int, d int)") + testKit.MustExec("insert into t value(1,5,4,4),(3,4,1,8),(4,2,6,10),(6,7,2,5),(7,1,4,9),(8,9,8,3),(9,1,9,1),(10,6,6,2)") + testKit.MustExec("alter table t add index (b)") + testKit.MustExec("alter table t add index (d)") + testKit.MustExec(`analyze table t`) + + ctx := context.Background() + h := dom.StatsHandle() + tb, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + statsTbl := h.GetTableStats(tblInfo) + + var ( + input []string + output []struct { + SQL string + Selectivity float64 + } + ) + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, tt) + require.NoErrorf(t, err, "error %v, for sql %s", err, tt) + require.Len(t, stmts, 1) + + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoErrorf(t, err, "error %v, for sql %s", err, tt) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoErrorf(t, err, "error %v, for building plan, sql %s", err, tt) + + sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + ds := sel.Children()[0].(*plannercore.DataSource) + + histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) + + ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) + require.NoErrorf(t, err, "error %v, for expr %s", err, tt) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Selectivity = ratio + }) + require.Truef(t, math.Abs(ratio-output[i].Selectivity) < eps, "for %s, needed: %v, got: %v", tt, output[i].Selectivity, ratio) + } + + // Test issue 19981 + testKit.MustExec("select * from t where _tidb_rowid is null or _tidb_rowid > 7") + + // Test issue 22134 + // Information about column n will not be in stats immediately after this SQL executed. + // If we don't have a check against this, DNF condition could lead to infinite recursion in Selectivity(). + testKit.MustExec("alter table t add column n timestamp;") + testKit.MustExec("select * from t where n = '2000-01-01' or n = '2000-01-02';") + + // Test issue 27294 + testKit.MustExec("create table tt (COL1 blob DEFAULT NULL,COL2 decimal(37,4) DEFAULT NULL,COL3 timestamp NULL DEFAULT NULL,COL4 int(11) DEFAULT NULL,UNIQUE KEY U_M_COL4(COL1(10),COL2), UNIQUE KEY U_M_COL5(COL3,COL2));") + testKit.MustExec("explain select * from tt where col1 is not null or col2 not between 454623814170074.2771 and -975540642273402.9269 and col3 not between '2039-1-19 10:14:57' and '2002-3-27 14:40:23';") +} + +func TestIndexEstimationCrossValidate(t *testing.T) { + domain.RunAutoAnalyze = false + 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,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)`)) + 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")) + + // Test issue 22466 + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(a int, b int, key b(b))") + tk.MustExec("insert into t2 values(1, 1), (2, 2), (3, 3), (4, 4), (5,5)") + // This line of select will mark column b stats as needed, and an invalid(empty) stats for column b + // will be loaded at the next analyze line, this will trigger the bug. + tk.MustQuery("select * from t2 where b=2") + tk.MustExec("analyze table t2 index b") + tk.MustQuery("explain select * from t2 where b=2").Check(testkit.Rows( + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.t2.b, 2)", + " └─TableFullScan_5 5.00 cop[tikv] table:t2 keep order:false")) +} + +func TestRangeStepOverflow(t *testing.T) { + domain.RunAutoAnalyze = false + 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 (col datetime)") + tk.MustExec("insert into t values('3580-05-26 07:16:48'),('4055-03-06 22:27:16'),('4862-01-26 07:16:54')") + h := dom.StatsHandle() + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + tk.MustExec("analyze table t") + // Trigger the loading of column stats. + tk.MustQuery("select * from t where col between '8499-1-23 2:14:38' and '9961-7-23 18:35:26'").Check(testkit.Rows()) + require.Nil(t, h.LoadNeededHistograms()) + // Must execute successfully after loading the column stats. + tk.MustQuery("select * from t where col between '8499-1-23 2:14:38' and '9961-7-23 18:35:26'").Check(testkit.Rows()) +} + +func TestSmallRangeEstimation(t *testing.T) { + domain.RunAutoAnalyze = false + 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)") + for i := 0; i < 400; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%v), (%v), (%v)", i, i, i)) // [0, 400) + } + testKit.MustExec("analyze table t with 0 topn") + + h := dom.StatsHandle() + table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + statsTbl := h.GetTableStats(table.Meta()) + sc := &stmtctx.StatementContext{} + col := statsTbl.Columns[table.Meta().Columns[0].ID] + + var input []struct { + Start int64 + End int64 + } + var output []struct { + Start int64 + End int64 + Count float64 + } + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) + for i, ran := range input { + count, err := col.GetColumnRowCount(sc, getRange(ran.Start, ran.End), statsTbl.Count, false) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].Start = ran.Start + output[i].End = ran.End + output[i].Count = count + }) + require.Truef(t, math.Abs(count-output[i].Count) < eps, "for [%v, %v], needed: around %v, got: %v", ran.Start, ran.End, output[i].Count, count) + } +} diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index ce1bc2f1cbee5..d77bfa2d397ef 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -15,25 +15,16 @@ package statistics_test import ( - "context" - "fmt" "math" "testing" "time" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/model" "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/stmtctx" - "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/testkit/testdata" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/ranger" @@ -130,161 +121,6 @@ func prepareSelectivity(testKit *testkit.TestKit, dom *domain.Domain) (*statisti return statsTbl, nil } -func TestSelectivity(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - testKit := testkit.NewTestKit(t, store) - statsTbl, err := prepareSelectivity(testKit, dom) - require.NoError(t, err) - longExpr := "0 < a and a = 1 " - for i := 1; i < 64; i++ { - longExpr += fmt.Sprintf(" and a > %d ", i) - } - tests := []struct { - exprs string - selectivity float64 - selectivityAfterIncrease float64 - }{ - { - exprs: "a > 0 and a < 2", - selectivity: 0.01851851851, - selectivityAfterIncrease: 0.01851851851, - }, - { - exprs: "a >= 1 and a < 2", - selectivity: 0.01851851851, - selectivityAfterIncrease: 0.01851851851, - }, - { - exprs: "a >= 1 and b > 1 and a < 2", - selectivity: 0.01783264746, - selectivityAfterIncrease: 0.01851851852, - }, - { - exprs: "a >= 1 and c > 1 and a < 2", - selectivity: 0.00617283950, - selectivityAfterIncrease: 0.00617283950, - }, - { - exprs: "a >= 1 and c >= 1 and a < 2", - selectivity: 0.01234567901, - selectivityAfterIncrease: 0.01234567901, - }, - { - exprs: "d = 0 and e = 1", - selectivity: 0.11111111111, - selectivityAfterIncrease: 0.11111111111, - }, - { - exprs: "b > 1", - selectivity: 0.96296296296, - selectivityAfterIncrease: 1, - }, - { - exprs: "a > 1 and b < 2 and c > 3 and d < 4 and e > 5", - selectivity: 0, - selectivityAfterIncrease: 0, - }, - { - exprs: longExpr, - selectivity: 0.001, - selectivityAfterIncrease: 0.001, - }, - } - - ctx := context.Background() - for _, tt := range tests { - sql := "select * from t where " + tt.exprs - sctx := testKit.Session().(sessionctx.Context) - stmts, err := session.Parse(sctx, sql) - require.NoErrorf(t, err, "for %s", tt.exprs) - require.Len(t, stmts, 1) - - ret := &plannercore.PreprocessorReturn{} - err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) - require.NoErrorf(t, err, "for expr %s", tt.exprs) - p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) - require.NoErrorf(t, err, "for building plan, expr %s", err, tt.exprs) - - sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) - ds := sel.Children()[0].(*plannercore.DataSource) - - histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) - - ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) - require.NoErrorf(t, err, "for %s", tt.exprs) - require.Truef(t, math.Abs(ratio-tt.selectivity) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio) - - histColl.Count *= 10 - ratio, _, err = histColl.Selectivity(sctx, sel.Conditions, nil) - require.NoErrorf(t, err, "for %s", tt.exprs) - require.Truef(t, math.Abs(ratio-tt.selectivityAfterIncrease) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivityAfterIncrease, ratio) - } -} - -// TestDiscreteDistribution tests the estimation for discrete data distribution. This is more common when the index -// consists several columns, and the first column has small NDV. -func TestDiscreteDistribution(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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 char(10), b int, key idx(a, b))") - for i := 0; i < 499; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values ('cn', %d)", i)) - } - for i := 0; i < 10; i++ { - testKit.MustExec("insert into t values ('tw', 0)") - } - testKit.MustExec("analyze table t") - var ( - input []string - output [][]string - ) - - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - - for i, tt := range input { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) - }) - testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) - } -} - -func TestSelectCombinedLowBound(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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(id int auto_increment, kid int, pid int, primary key(id), key(kid, pid))") - testKit.MustExec("insert into t (kid, pid) values (1,2), (1,3), (1,4),(1, 11), (1, 12), (1, 13), (1, 14), (2, 2), (2, 3), (2, 4)") - testKit.MustExec("analyze table t") - var ( - input []string - output [][]string - ) - - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - - for i, tt := range input { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) - }) - testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) - } -} - func getRange(start, end int64) []*ranger.Range { ran := &ranger.Range{ LowVal: []types.Datum{types.NewIntDatum(start)}, @@ -293,377 +129,6 @@ func getRange(start, end int64) []*ranger.Range { return []*ranger.Range{ran} } -func TestOutOfRangeEstimation(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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 unsigned)") - for i := 0; i < 3000; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%v)", i/5+300)) // [300, 900) - } - testKit.MustExec("analyze table t with 2000 samples") - - h := dom.StatsHandle() - table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - statsTbl := h.GetTableStats(table.Meta()) - sc := &stmtctx.StatementContext{} - col := statsTbl.Columns[table.Meta().Columns[0].ID] - count, err := col.GetColumnRowCount(sc, getRange(900, 900), statsTbl.Count, false) - require.NoError(t, err) - // Because the ANALYZE collect data by random sampling, so the result is not an accurate value. - // so we use a range here. - require.Truef(t, count < 5.5, "expected: around 5.0, got: %v", count) - require.Truef(t, count > 4.5, "expected: around 5.0, got: %v", count) - - var input []struct { - Start int64 - End int64 - } - var output []struct { - Start int64 - End int64 - Count float64 - } - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - increasedTblRowCount := int64(float64(statsTbl.Count) * 1.5) - for i, ran := range input { - count, err = col.GetColumnRowCount(sc, getRange(ran.Start, ran.End), increasedTblRowCount, false) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].Start = ran.Start - output[i].End = ran.End - output[i].Count = count - }) - require.Truef(t, count < output[i].Count*1.2, "for [%v, %v], needed: around %v, got: %v", ran.Start, ran.End, output[i].Count, count) - require.Truef(t, count > output[i].Count*0.8, "for [%v, %v], needed: around %v, got: %v", ran.Start, ran.End, output[i].Count, count) - } -} - -func TestEstimationForUnknownValues(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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, key idx(a, b))") - testKit.MustExec("set @@tidb_analyze_version=1") - testKit.MustExec("analyze table t") - for i := 0; i < 10; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) - } - h := dom.StatsHandle() - require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - for i := 0; i < 10; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i+10, i+10)) - } - require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.Nil(t, h.Update(dom.InfoSchema())) - table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - statsTbl := h.GetTableStats(table.Meta()) - - sc := &stmtctx.StatementContext{} - colID := table.Meta().Columns[0].ID - count, err := statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(30, 30)) - require.NoError(t, err) - require.Equal(t, 0.2, count) - - count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(9, 30)) - require.NoError(t, err) - require.Equal(t, 7.2, count) - - count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(9, math.MaxInt64)) - require.NoError(t, err) - require.Equal(t, 7.2, count) - - idxID := table.Meta().Indices[0].ID - count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(30, 30)) - require.NoError(t, err) - require.Equal(t, 0.1, count) - - count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(9, 30)) - require.NoError(t, err) - require.Equal(t, 7.0, count) - - testKit.MustExec("truncate table t") - testKit.MustExec("insert into t values (null, null)") - testKit.MustExec("analyze table t") - table, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - statsTbl = h.GetTableStats(table.Meta()) - - colID = table.Meta().Columns[0].ID - count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(1, 30)) - require.NoError(t, err) - require.Equal(t, 0.0, count) - - testKit.MustExec("drop table t") - testKit.MustExec("create table t(a int, b int, index idx(b))") - testKit.MustExec("insert into t values (1,1)") - testKit.MustExec("analyze table t") - table, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - statsTbl = h.GetTableStats(table.Meta()) - - colID = table.Meta().Columns[0].ID - count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(2, 2)) - require.NoError(t, err) - require.Equal(t, 0.0, count) - - idxID = table.Meta().Indices[0].ID - count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(2, 2)) - require.NoError(t, err) - require.Equal(t, 0.0, count) -} - -func TestEstimationUniqueKeyEqualConds(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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, c int, unique key(b))") - testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7)") - testKit.MustExec("analyze table t with 4 cmsketch width, 1 cmsketch depth;") - table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - statsTbl := dom.StatsHandle().GetTableStats(table.Meta()) - - sc := &stmtctx.StatementContext{} - idxID := table.Meta().Indices[0].ID - count, err := statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(7, 7)) - require.NoError(t, err) - require.Equal(t, 1.0, count) - - count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(6, 6)) - require.NoError(t, err) - require.Equal(t, 1.0, count) - - colID := table.Meta().Columns[0].ID - count, err = statsTbl.GetRowCountByIntColumnRanges(sc, colID, getRange(7, 7)) - require.NoError(t, err) - require.Equal(t, 1.0, count) - - count, err = statsTbl.GetRowCountByIntColumnRanges(sc, colID, getRange(6, 6)) - require.NoError(t, err) - require.Equal(t, 1.0, count) -} - -func TestPrimaryKeySelectivity(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - store, clean := testkit.CreateMockStore(t) - defer clean() - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - testKit.MustExec("create table t(a char(10) primary key, b int)") - var input, output [][]string - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - for i, ts := range input { - for j, tt := range ts { - if j != len(ts)-1 { - testKit.MustExec(tt) - } - testdata.OnRecord(func() { - if j == len(ts)-1 { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) - } - }) - if j == len(ts)-1 { - testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) - } - } - } -} - -func TestStatsVer2(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - store, clean := testkit.CreateMockStore(t) - defer clean() - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set tidb_analyze_version=2") - - testKit.MustExec("drop table if exists tint") - testKit.MustExec("create table tint(a int, b int, c int, index singular(a), index multi(b, c))") - testKit.MustExec("insert into tint 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)") - testKit.MustExec("analyze table tint with 2 topn, 3 buckets") - - testKit.MustExec("drop table if exists tdouble") - testKit.MustExec("create table tdouble(a double, b double, c double, index singular(a), index multi(b, c))") - testKit.MustExec("insert into tdouble 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)") - testKit.MustExec("analyze table tdouble with 2 topn, 3 buckets") - - testKit.MustExec("drop table if exists tdecimal") - testKit.MustExec("create table tdecimal(a decimal(40, 20), b decimal(40, 20), c decimal(40, 20), index singular(a), index multi(b, c))") - testKit.MustExec("insert into tdecimal 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)") - testKit.MustExec("analyze table tdecimal with 2 topn, 3 buckets") - - testKit.MustExec("drop table if exists tstring") - testKit.MustExec("create table tstring(a varchar(64), b varchar(64), c varchar(64), index singular(a), index multi(b, c))") - testKit.MustExec("insert into tstring 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')") - testKit.MustExec("analyze table tstring with 2 topn, 3 buckets") - - testKit.MustExec("drop table if exists tdatetime") - testKit.MustExec("create table tdatetime(a datetime, b datetime, c datetime, index singular(a), index multi(b, c))") - testKit.MustExec("insert into tdatetime values ('2001-01-01', '2001-01-01', '2001-01-01'), ('2001-01-02', '2001-01-02', '2001-01-02'), ('2001-01-03', '2001-01-03', '2001-01-03'), ('2001-01-04', '2001-01-04', '2001-01-04')") - testKit.MustExec("analyze table tdatetime with 2 topn, 3 buckets") - - testKit.MustExec("drop table if exists tprefix") - testKit.MustExec("create table tprefix(a varchar(64), b varchar(64), index prefixa(a(2)))") - testKit.MustExec("insert into tprefix values ('111', '111'), ('222', '222'), ('333', '333'), ('444', '444'), ('555', '555'), ('666', '666')") - testKit.MustExec("analyze table tprefix with 2 topn, 3 buckets") - - // test with clustered index - testKit.MustExec("drop table if exists ct1") - testKit.MustExec("create table ct1 (a int, pk varchar(10), primary key(pk) clustered)") - testKit.MustExec("insert into ct1 values (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'), (7, '7'), (8, '8')") - testKit.MustExec("analyze table ct1 with 2 topn, 3 buckets") - - testKit.MustExec("drop table if exists ct2") - testKit.MustExec("create table ct2 (a int, b int, c int, primary key(a, b) clustered)") - testKit.MustExec("insert into ct2 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)") - testKit.MustExec("analyze table ct2 with 2 topn, 3 buckets") - - rows := testKit.MustQuery("select stats_ver from mysql.stats_histograms").Rows() - for _, r := range rows { - // ensure statsVer = 2 - require.Equal(t, "2", fmt.Sprintf("%v", r[0])) - } - - var ( - input []string - output [][]string - ) - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - for i := range input { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) - }) - testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) - } -} - -func TestTopNOutOfHist(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - store, clean := testkit.CreateMockStore(t) - defer clean() - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set tidb_analyze_version=2") - - testKit.MustExec("drop table if exists topn_before_hist") - testKit.MustExec("create table topn_before_hist(a int, index idx(a))") - testKit.MustExec("insert into topn_before_hist values(1), (1), (1), (1), (3), (3), (4), (5), (6)") - testKit.MustExec("analyze table topn_before_hist with 2 topn, 3 buckets") - - testKit.MustExec("create table topn_after_hist(a int, index idx(a))") - testKit.MustExec("insert into topn_after_hist values(2), (2), (3), (4), (5), (7), (7), (7), (7)") - testKit.MustExec("analyze table topn_after_hist with 2 topn, 3 buckets") - - testKit.MustExec("create table topn_before_hist_no_index(a int)") - testKit.MustExec("insert into topn_before_hist_no_index values(1), (1), (1), (1), (3), (3), (4), (5), (6)") - testKit.MustExec("analyze table topn_before_hist_no_index with 2 topn, 3 buckets") - - testKit.MustExec("create table topn_after_hist_no_index(a int)") - testKit.MustExec("insert into topn_after_hist_no_index values(2), (2), (3), (4), (5), (7), (7), (7), (7)") - testKit.MustExec("analyze table topn_after_hist_no_index with 2 topn, 3 buckets") - - var ( - input []string - output [][]string - ) - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - for i := range input { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) - }) - testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) - } -} - -func TestColumnIndexNullEstimation(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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, c int, index idx_b(b), index idx_c_a(c, a))") - testKit.MustExec("insert into t values(1,null,1),(2,null,2),(3,3,3),(4,null,4),(null,null,null);") - h := dom.StatsHandle() - require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - var ( - input []string - output [][]string - ) - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - for i := 0; i < 5; i++ { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) - }) - testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) - } - // Make sure column stats has been loaded. - testKit.MustExec(`explain select * from t where a is null`) - require.Nil(t, h.LoadNeededHistograms()) - for i := 5; i < len(input); i++ { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) - }) - testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) - } -} - -func TestUniqCompEqualEst(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - testKit.MustExec("drop table if exists t") - testKit.MustExec("create table t(a int, b int, primary key(a, b))") - testKit.MustExec("insert into t values(1,1),(1,2),(1,3),(1,4),(1,5),(1,6),(1,7),(1,8),(1,9),(1,10)") - h := dom.StatsHandle() - require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - var ( - input []string - output [][]string - ) - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - for i := 0; i < 1; i++ { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) - }) - testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) - } -} - func TestSelectivityGreedyAlgo(t *testing.T) { t.Parallel() nodes := make([]*statistics.StatsNode, 3) @@ -682,170 +147,3 @@ func TestSelectivityGreedyAlgo(t *testing.T) { require.Equal(t, 1, len(usedSets)) require.Equal(t, int64(1), usedSets[0].ID) } - -// TestDNFCondSelectivity tests selectivity calculation with DNF conditions covered by using independence assumption. -func TestDNFCondSelectivity(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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, c int, d int)") - testKit.MustExec("insert into t value(1,5,4,4),(3,4,1,8),(4,2,6,10),(6,7,2,5),(7,1,4,9),(8,9,8,3),(9,1,9,1),(10,6,6,2)") - testKit.MustExec("alter table t add index (b)") - testKit.MustExec("alter table t add index (d)") - testKit.MustExec(`analyze table t`) - - ctx := context.Background() - h := dom.StatsHandle() - tb, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := tb.Meta() - statsTbl := h.GetTableStats(tblInfo) - - var ( - input []string - output []struct { - SQL string - Selectivity float64 - } - ) - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - for i, tt := range input { - sctx := testKit.Session().(sessionctx.Context) - stmts, err := session.Parse(sctx, tt) - require.NoErrorf(t, err, "error %v, for sql %s", err, tt) - require.Len(t, stmts, 1) - - ret := &plannercore.PreprocessorReturn{} - err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) - require.NoErrorf(t, err, "error %v, for sql %s", err, tt) - p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) - require.NoErrorf(t, err, "error %v, for building plan, sql %s", err, tt) - - sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) - ds := sel.Children()[0].(*plannercore.DataSource) - - histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) - - ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) - require.NoErrorf(t, err, "error %v, for expr %s", err, tt) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Selectivity = ratio - }) - require.Truef(t, math.Abs(ratio-output[i].Selectivity) < eps, "for %s, needed: %v, got: %v", tt, output[i].Selectivity, ratio) - } - - // Test issue 19981 - testKit.MustExec("select * from t where _tidb_rowid is null or _tidb_rowid > 7") - - // Test issue 22134 - // Information about column n will not be in stats immediately after this SQL executed. - // If we don't have a check against this, DNF condition could lead to infinite recursion in Selectivity(). - testKit.MustExec("alter table t add column n timestamp;") - testKit.MustExec("select * from t where n = '2000-01-01' or n = '2000-01-02';") - - // Test issue 27294 - testKit.MustExec("create table tt (COL1 blob DEFAULT NULL,COL2 decimal(37,4) DEFAULT NULL,COL3 timestamp NULL DEFAULT NULL,COL4 int(11) DEFAULT NULL,UNIQUE KEY U_M_COL4(COL1(10),COL2), UNIQUE KEY U_M_COL5(COL3,COL2));") - testKit.MustExec("explain select * from tt where col1 is not null or col2 not between 454623814170074.2771 and -975540642273402.9269 and col3 not between '2039-1-19 10:14:57' and '2002-3-27 14:40:23';") -} - -func TestIndexEstimationCrossValidate(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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,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)`)) - 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")) - - // Test issue 22466 - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2(a int, b int, key b(b))") - tk.MustExec("insert into t2 values(1, 1), (2, 2), (3, 3), (4, 4), (5,5)") - // This line of select will mark column b stats as needed, and an invalid(empty) stats for column b - // will be loaded at the next analyze line, this will trigger the bug. - tk.MustQuery("select * from t2 where b=2") - tk.MustExec("analyze table t2 index b") - tk.MustQuery("explain select * from t2 where b=2").Check(testkit.Rows( - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.t2.b, 2)", - " └─TableFullScan_5 5.00 cop[tikv] table:t2 keep order:false")) -} - -func TestRangeStepOverflow(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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 (col datetime)") - tk.MustExec("insert into t values('3580-05-26 07:16:48'),('4055-03-06 22:27:16'),('4862-01-26 07:16:54')") - h := dom.StatsHandle() - require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - tk.MustExec("analyze table t") - // Trigger the loading of column stats. - tk.MustQuery("select * from t where col between '8499-1-23 2:14:38' and '9961-7-23 18:35:26'").Check(testkit.Rows()) - require.Nil(t, h.LoadNeededHistograms()) - // Must execute successfully after loading the column stats. - tk.MustQuery("select * from t where col between '8499-1-23 2:14:38' and '9961-7-23 18:35:26'").Check(testkit.Rows()) -} - -func TestSmallRangeEstimation(t *testing.T) { - t.Parallel() - domain.RunAutoAnalyze = false - 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)") - for i := 0; i < 400; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%v), (%v), (%v)", i, i, i)) // [0, 400) - } - testKit.MustExec("analyze table t with 0 topn") - - h := dom.StatsHandle() - table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - statsTbl := h.GetTableStats(table.Meta()) - sc := &stmtctx.StatementContext{} - col := statsTbl.Columns[table.Meta().Columns[0].ID] - - var input []struct { - Start int64 - End int64 - } - var output []struct { - Start int64 - End int64 - Count float64 - } - statsSuiteData := statistics.GetStatsSuiteData() - statsSuiteData.GetTestCases(t, &input, &output) - for i, ran := range input { - count, err := col.GetColumnRowCount(sc, getRange(ran.Start, ran.End), statsTbl.Count, false) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].Start = ran.Start - output[i].End = ran.End - output[i].Count = count - }) - require.Truef(t, math.Abs(count-output[i].Count) < eps, "for [%v, %v], needed: around %v, got: %v", ran.Start, ran.End, output[i].Count, count) - } -} diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index bd6ca78a01928..a3c385f39df6e 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -204,9 +204,7 @@ func (s *tikvStore) Describe() string { return "TiKV is a distributed transactional key-value database" } -var ( - ldflagGetEtcdAddrsFromConfig = "0" // 1:Yes, otherwise:No -) +var ldflagGetEtcdAddrsFromConfig = "0" // 1:Yes, otherwise:No const getAllMembersBackoff = 5000 @@ -300,17 +298,8 @@ func (s *tikvStore) GetMemCache() kv.MemManager { } // Begin a global transaction. -func (s *tikvStore) Begin() (kv.Transaction, error) { - txn, err := s.KVStore.Begin() - if err != nil { - return nil, derr.ToTiDBErr(err) - } - return txn_driver.NewTiKVTxn(txn), err -} - -// BeginWithOption begins a transaction with given option -func (s *tikvStore) BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) { - txn, err := s.KVStore.BeginWithOption(option) +func (s *tikvStore) Begin(opts ...tikv.TxnOption) (kv.Transaction, error) { + txn, err := s.KVStore.Begin(opts...) if err != nil { return nil, derr.ToTiDBErr(err) } diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 1a8da7c96ad05..293f57755ed77 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -18,6 +18,7 @@ import ( "bytes" "container/heap" "context" + "encoding/hex" "encoding/json" "fmt" "math" @@ -45,7 +46,9 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" tikverr "github.com/tikv/client-go/v2/error" tikvstore "github.com/tikv/client-go/v2/kv" @@ -703,6 +706,9 @@ func (w *GCWorker) deleteRanges(ctx context.Context, safePoint uint64, concurren startKey, endKey := r.Range() err = w.doUnsafeDestroyRangeRequest(ctx, startKey, endKey, concurrency) + failpoint.Inject("ignoreDeleteRangeFailed", func() { + err = nil + }) if err != nil { logutil.Logger(ctx).Error("[gc worker] delete range failed on range", zap.String("uuid", w.uuid), @@ -1938,17 +1944,51 @@ func (w *GCWorker) doGCLabelRules(dr util.DelRangeTask) (err error) { startKey kv.Key physicalTableIDs []int64 ruleIDs []string + rules map[string]*label.Rule ) if err = historyJob.DecodeArgs(&startKey, &physicalTableIDs, &ruleIDs); err != nil { return } + // TODO: Here we need to get rules from PD and filter the rules which is not elegant. We should find a better way. + rules, err = infosync.GetLabelRules(context.TODO(), ruleIDs) + if err != nil { + return + } + + ruleIDs = getGCRules(append(physicalTableIDs, historyJob.TableID), rules) patch := label.NewRulePatch([]*label.Rule{}, ruleIDs) err = infosync.UpdateLabelRules(context.TODO(), patch) } return } +func getGCRules(ids []int64, rules map[string]*label.Rule) []string { + oldRange := make(map[string]struct{}) + for _, id := range ids { + startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(id))) + endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(id+1))) + oldRange[startKey+endKey] = struct{}{} + } + + var gcRules []string + for _, rule := range rules { + find := false + for _, d := range rule.Data { + if r, ok := d.(map[string]interface{}); ok { + nowRange := fmt.Sprintf("%s%s", r["start_key"], r["end_key"]) + if _, ok := oldRange[nowRange]; ok { + find = true + } + } + } + if find { + gcRules = append(gcRules, rule.ID) + } + } + return gcRules +} + // RunGCJob sends GC command to KV. It is exported for kv api, do not use it with GCWorker at the same time. func RunGCJob(ctx context.Context, s tikv.Storage, pd pd.Client, safePoint uint64, identifier string, concurrency int) error { gcWorker := &GCWorker{ @@ -2053,6 +2093,7 @@ func NewMockGCWorker(store kv.Storage) (*MockGCWorker, error) { gcIsRunning: false, lastFinish: time.Now(), done: make(chan error), + pdClient: store.(tikv.Storage).GetRegionCache().PDClient(), } return &MockGCWorker{worker: worker}, nil } diff --git a/store/helper/helper.go b/store/helper/helper.go index 80f0ed21c7a60..3b8e122071959 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -51,8 +51,7 @@ import ( // Storage represents a storage that connects TiKV. // Methods copied from kv.Storage and tikv.Storage due to limitation of go1.13. type Storage interface { - Begin() (kv.Transaction, error) - BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) + Begin(opts ...tikv.TxnOption) (kv.Transaction, error) GetSnapshot(ver kv.Version) kv.Snapshot GetClient() kv.Client GetMPPClient() kv.MPPClient diff --git a/store/mockstore/mockcopr/cop_handler_dag.go b/store/mockstore/mockcopr/cop_handler_dag.go index b63a020b78ed8..4d71a1caa3994 100644 --- a/store/mockstore/mockcopr/cop_handler_dag.go +++ b/store/mockstore/mockcopr/cop_handler_dag.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" @@ -921,12 +922,14 @@ func extractOffsetsInExpr(expr *tipb.Expr, columns []*tipb.ColumnInfo, collector // fieldTypeFromPBColumn creates a types.FieldType from tipb.ColumnInfo. func fieldTypeFromPBColumn(col *tipb.ColumnInfo) *types.FieldType { + charsetStr, collationStr, _ := charset.GetCharsetInfoByID(int(collate.RestoreCollationIDIfNeeded(col.GetCollation()))) return &types.FieldType{ Tp: byte(col.GetTp()), Flag: uint(col.Flag), Flen: int(col.GetColumnLen()), Decimal: int(col.GetDecimal()), Elems: col.Elems, - Collate: collate.CollationID2Name(collate.RestoreCollationIDIfNeeded(col.GetCollation())), + Charset: charsetStr, + Collate: collationStr, } } diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 83a40661e24d3..a85b46166631f 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -74,8 +74,8 @@ func (s *mockStorage) Describe() string { } // Begin a global transaction. -func (s *mockStorage) Begin() (kv.Transaction, error) { - txn, err := s.KVStore.Begin() +func (s *mockStorage) Begin(opts ...tikv.TxnOption) (kv.Transaction, error) { + txn, err := s.KVStore.Begin(opts...) return newTiKVTxn(txn, err) } @@ -84,11 +84,6 @@ func (s *mockStorage) ShowStatus(ctx context.Context, key string) (interface{}, return nil, kv.ErrNotImplemented } -// BeginWithOption begins a transaction with given option -func (s *mockStorage) BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) { - return newTiKVTxn(s.KVStore.BeginWithOption(option)) -} - // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. // if ver is MaxVersion or > current max committed version, we will use current version for this snapshot. func (s *mockStorage) GetSnapshot(ver kv.Version) kv.Snapshot { diff --git a/store/mockstore/unistore/cophandler/cop_handler.go b/store/mockstore/unistore/cophandler/cop_handler.go index 504741ece2125..fa7b784126e3e 100644 --- a/store/mockstore/unistore/cophandler/cop_handler.go +++ b/store/mockstore/unistore/cophandler/cop_handler.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" @@ -445,13 +446,15 @@ func appendRow(chunks []tipb.Chunk, data []byte, rowCnt int) []tipb.Chunk { // fieldTypeFromPBColumn creates a types.FieldType from tipb.ColumnInfo. func fieldTypeFromPBColumn(col *tipb.ColumnInfo) *types.FieldType { + charsetStr, collationStr, _ := charset.GetCharsetInfoByID(int(collate.RestoreCollationIDIfNeeded(col.GetCollation()))) return &types.FieldType{ Tp: byte(col.GetTp()), Flag: uint(col.Flag), Flen: int(col.GetColumnLen()), Decimal: int(col.GetDecimal()), Elems: col.Elems, - Collate: collate.CollationID2Name(collate.RestoreCollationIDIfNeeded(col.GetCollation())), + Charset: charsetStr, + Collate: collationStr, } } diff --git a/table/table.go b/table/table.go index c8717686a9428..f39e9b2d9fa8d 100644 --- a/table/table.go +++ b/table/table.go @@ -252,17 +252,10 @@ var MockTableFromMeta func(tableInfo *model.TableInfo) Table type CachedTable interface { Table - // TryGetMemcache Check if the cache table is readable, if it is readable, - // Return the pointer to the MemBuffer and true otherwise return nil and false - TryGetMemcache(ts uint64) (kv.MemBuffer, bool) + // TryReadFromCache checks if the cache table is readable. + TryReadFromCache(ts uint64) kv.MemBuffer // UpdateLockForRead If you cannot meet the conditions of the read buffer, // you need to update the lock information and read the data from the original table UpdateLockForRead(store kv.Storage, ts uint64) error } - -// CacheData pack the cache data and lease -type CacheData struct { - Lease uint64 - kv.MemBuffer -} diff --git a/table/tables/cache.go b/table/tables/cache.go index f472ce06fafde..ba8786a65bae3 100644 --- a/table/tables/cache.go +++ b/table/tables/cache.go @@ -29,8 +29,10 @@ import ( "github.com/tikv/client-go/v2/tikv" ) -var _ table.Table = &cachedTable{} -var _ table.CachedTable = &cachedTable{} +var ( + _ table.Table = &cachedTable{} + _ table.CachedTable = &cachedTable{} +) type cachedTable struct { TableCommon @@ -38,6 +40,13 @@ type cachedTable struct { handle StateRemote } +// cacheData pack the cache data and lease. +type cacheData struct { + Start uint64 + Lease uint64 + kv.MemBuffer +} + func leaseFromTS(ts uint64) uint64 { // TODO make this configurable in the following PRs const defaultLeaseDuration time.Duration = 3 * time.Second @@ -49,23 +58,23 @@ func leaseFromTS(ts uint64) uint64 { func newMemBuffer(store kv.Storage) (kv.MemBuffer, error) { // Here is a trick to get a MemBuffer data, because the internal API is not exposed. // Create a transaction with start ts 0, and take the MemBuffer out. - buffTxn, err := store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTS(0)) + buffTxn, err := store.Begin(tikv.WithStartTS(0)) if err != nil { return nil, err } return buffTxn.GetMemBuffer(), nil } -func (c *cachedTable) TryGetMemcache(ts uint64) (kv.MemBuffer, bool) { +func (c *cachedTable) TryReadFromCache(ts uint64) kv.MemBuffer { tmp := c.cacheData.Load() if tmp == nil { - return nil, false + return nil } - data := tmp.(*table.CacheData) - if data.Lease > ts { - return data.MemBuffer, true + data := tmp.(*cacheData) + if ts >= data.Start && ts < data.Lease { + return data } - return nil, false + return nil } var mockStateRemote = struct { @@ -88,42 +97,45 @@ func NewCachedTable(tbl *TableCommon) (table.Table, error) { return ret, nil } -func (c *cachedTable) loadDataFromOriginalTable(store kv.Storage, lease uint64) (kv.MemBuffer, error) { +func (c *cachedTable) loadDataFromOriginalTable(store kv.Storage, lease uint64) (kv.MemBuffer, uint64, error) { buffer, err := newMemBuffer(store) if err != nil { - return nil, err + return nil, 0, err } - err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { + var startTS uint64 + err = kv.RunInNewTxn(context.Background(), store, true, func(ctx context.Context, txn kv.Transaction) error { prefix := tablecodec.GenTablePrefix(c.tableID) if err != nil { - return err + return errors.Trace(err) } - if txn.StartTS() >= lease { + startTS = txn.StartTS() + if startTS >= lease { return errors.New("the loaded data is outdated for caching") } it, err := txn.Iter(prefix, prefix.PrefixNext()) if err != nil { - return err + return errors.Trace(err) } defer it.Close() + for it.Valid() && it.Key().HasPrefix(prefix) { value := it.Value() err = buffer.Set(it.Key(), value) if err != nil { - return err + return errors.Trace(err) } err = it.Next() if err != nil { - return err + return errors.Trace(err) } } return nil }) if err != nil { - return nil, err + return nil, 0, err } - return buffer, nil + return buffer, startTS, nil } func (c *cachedTable) UpdateLockForRead(store kv.Storage, ts uint64) error { @@ -135,12 +147,13 @@ func (c *cachedTable) UpdateLockForRead(store kv.Storage, ts uint64) error { return errors.Trace(err) } if succ { - mb, err := c.loadDataFromOriginalTable(store, lease) + mb, startTS, err := c.loadDataFromOriginalTable(store, lease) if err != nil { return errors.Trace(err) } - c.cacheData.Store(&table.CacheData{ + c.cacheData.Store(&cacheData{ + Start: startTS, Lease: lease, MemBuffer: mb, }) @@ -161,7 +174,6 @@ func (c *cachedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts .. return nil, errors.Trace(err) } return c.TableCommon.AddRecord(ctx, r, opts...) - } // UpdateRecord implements table.Table diff --git a/table/tables/cache_test.go b/table/tables/cache_test.go index 15298764060e3..6441c0228709e 100644 --- a/table/tables/cache_test.go +++ b/table/tables/cache_test.go @@ -53,12 +53,15 @@ func TestCacheTableBasicScan(t *testing.T) { 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.Session().GetSessionVars().StmtCtx.CacheTableUsed() { + 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", @@ -71,10 +74,12 @@ func TestCacheTableBasicScan(t *testing.T) { // 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.Session().GetSessionVars().StmtCtx.CacheTableUsed() { + 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", @@ -86,15 +91,17 @@ func TestCacheTableBasicScan(t *testing.T) { " └─TableFullScan 10000.00 cop[tikv] table:join_t3 keep order:false, stats:pseudo")) // Second read will from cache table - for i := 0; i < 10; i++ { + 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.Session().GetSessionVars().StmtCtx.CacheTableUsed() { + 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", @@ -105,10 +112,12 @@ func TestCacheTableBasicScan(t *testing.T) { "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.Session().GetSessionVars().StmtCtx.CacheTableUsed() { + if tk.HasPlan("select /*+ use_index(tmp1, u) */ * from tmp1 where u>101 order by u", "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 ", @@ -123,7 +132,7 @@ func TestCacheTableBasicScan(t *testing.T) { )) tk.MustQuery("show warnings").Check(testkit.Rows()) - // For IndexMerge, temporary table should not use index merge + // 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", @@ -164,7 +173,6 @@ func TestCacheCondition(t *testing.T) { } func TestCacheTableBasicReadAndWrite(t *testing.T) { - t.Parallel() store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -204,7 +212,6 @@ func TestCacheTableBasicReadAndWrite(t *testing.T) { } func TestCacheTableComplexRead(t *testing.T) { - t.Parallel() store, clean := testkit.CreateMockStore(t) defer clean() doneCh := make(chan struct{}, 1) @@ -236,3 +243,127 @@ func TestCacheTableComplexRead(t *testing.T) { tk1.HasPlan("select *from complex_cache where id > 7", "UnionScan") tk1.MustExec("commit") } + +func TestBeginSleepABA(t *testing.T) { + // During the change "cache1 -> no cache -> cache2", + // cache1 and cache2 may be not the same anymore + // A transaction should not only check the cache exists, but also check the cache unchanged. + + store, clean := testkit.CreateMockStore(t) + defer clean() + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk2.MustExec("use test") + tk1.MustExec("drop table if exists aba") + tk1.MustExec("create table aba (id int, v int)") + tk1.MustExec("insert into aba values (1, 1)") + tk1.MustExec("alter table aba cache") + tk1.MustQuery("select * from aba").Check(testkit.Rows("1 1")) + + // Begin, read from cache. + tk1.MustExec("begin") + cacheUsed := false + for i := 0; i < 100; i++ { + if tk1.HasPlan("select * from aba", "UnionScan") { + cacheUsed = true + break + } + } + require.True(t, cacheUsed) + + // Another session change the data and make the cache unavailable. + tk2.MustExec("update aba set v = 2") + + // And then make the cache available again. + for i := 0; i < 50; i++ { + tk2.MustQuery("select * from aba").Check(testkit.Rows("1 2")) + if tk2.HasPlan("select * from aba", "UnionScan") { + cacheUsed = true + break + } + time.Sleep(100 * time.Millisecond) + } + 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")) +} + +func TestCacheTablePointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table cache_point (id int primary key auto_increment, u int unique, v int)") + tk.MustExec("insert into cache_point values(1, 11, 101)") + tk.MustExec("insert into cache_point values(2, 12, 102)") + tk.MustExec("alter table cache_point cache") + // check point get out transaction + tk.MustQuery("select * from cache_point where id=1").Check(testkit.Rows("1 11 101")) + tk.MustQuery("select * from cache_point where u=11").Check(testkit.Rows("1 11 101")) + tk.MustQuery("select * from cache_point where id=2").Check(testkit.Rows("2 12 102")) + tk.MustQuery("select * from cache_point where u=12").Check(testkit.Rows("2 12 102")) + tk.MustQuery("select * from cache_point where u > 10 and u < 12").Check(testkit.Rows("1 11 101")) + + // check point get in transaction + tk.MustExec("begin") + tk.MustQuery("select * from cache_point where id=1").Check(testkit.Rows("1 11 101")) + tk.MustQuery("select * from cache_point where u=11").Check(testkit.Rows("1 11 101")) + tk.MustQuery("select * from cache_point where id=2").Check(testkit.Rows("2 12 102")) + tk.MustQuery("select * from cache_point where u=12").Check(testkit.Rows("2 12 102")) + tk.MustExec("insert into cache_point values(3, 13, 103)") + tk.MustQuery("select * from cache_point where id=3").Check(testkit.Rows("3 13 103")) + tk.MustQuery("select * from cache_point where u=13").Check(testkit.Rows("3 13 103")) + tk.MustQuery("select * from cache_point where u > 12 and u < 14").Check(testkit.Rows("3 13 103")) + tk.MustExec("update cache_point set v=999 where id=2") + tk.MustQuery("select * from cache_point where id=2").Check(testkit.Rows("2 12 999")) + tk.MustExec("commit") + + // check point get after transaction + tk.MustQuery("select * from cache_point where id=3").Check(testkit.Rows("3 13 103")) + tk.MustQuery("select * from cache_point where u=13").Check(testkit.Rows("3 13 103")) + tk.MustQuery("select * from cache_point where id=2").Check(testkit.Rows("2 12 999")) +} + +func TestCacheTableBatchPointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table bp_cache_tmp1 (id int primary key auto_increment, u int unique, v int)") + tk.MustExec("insert into bp_cache_tmp1 values(1, 11, 101)") + tk.MustExec("insert into bp_cache_tmp1 values(2, 12, 102)") + tk.MustExec("insert into bp_cache_tmp1 values(3, 13, 103)") + tk.MustExec("insert into bp_cache_tmp1 values(4, 14, 104)") + + // check point get out transaction + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 3)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 3, 5)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13, 15)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13) and u in (12, 13)").Check(testkit.Rows("3 13 103")) + // check point get in transaction + tk.MustExec("begin") + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 3)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 3, 5)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13, 15)").Check(testkit.Rows("1 11 101", "3 13 103")) + tk.MustExec("insert into bp_cache_tmp1 values(6, 16, 106)") + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 6)").Check(testkit.Rows("1 11 101", "6 16 106")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 16)").Check(testkit.Rows("1 11 101", "6 16 106")) + tk.MustExec("update bp_cache_tmp1 set v=999 where id=3") + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 3)").Check(testkit.Rows("1 11 101", "3 13 999")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13)").Check(testkit.Rows("1 11 101", "3 13 999")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13) and u in (12, 13)").Check(testkit.Rows("3 13 999")) + tk.MustExec("delete from bp_cache_tmp1 where id=4") + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 4)").Check(testkit.Rows("1 11 101")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 14)").Check(testkit.Rows("1 11 101")) + tk.MustExec("commit") + + // check point get after transaction + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 3, 6)").Check(testkit.Rows("1 11 101", "3 13 999", "6 16 106")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 13, 16)").Check(testkit.Rows("1 11 101", "3 13 999", "6 16 106")) + tk.MustQuery("select * from bp_cache_tmp1 where id in (1, 4)").Check(testkit.Rows("1 11 101")) + tk.MustQuery("select * from bp_cache_tmp1 where u in (11, 14)").Check(testkit.Rows("1 11 101")) +} diff --git a/table/temptable/ddl.go b/table/temptable/ddl.go index 7a31b3690b885..fddd0bbbeabf5 100644 --- a/table/temptable/ddl.go +++ b/table/temptable/ddl.go @@ -146,7 +146,7 @@ func ensureSessionData(sctx sessionctx.Context) (variable.TemporaryTableData, er sessVars := sctx.GetSessionVars() if sessVars.TemporaryTableData == nil { // Create this txn just for getting a MemBuffer. It's a little tricky - bufferTxn, err := sctx.GetStore().BeginWithOption(tikv.DefaultStartTSOption().SetStartTS(0)) + bufferTxn, err := sctx.GetStore().Begin(tikv.WithStartTS(0)) if err != nil { return nil, err } diff --git a/util/hint/hint_processor.go b/util/hint/hint_processor.go index 9cbf41e93c0a2..be0d5bcd02a2b 100644 --- a/util/hint/hint_processor.go +++ b/util/hint/hint_processor.go @@ -32,6 +32,7 @@ import ( ) var supportedHintNameForInsertStmt = map[string]struct{}{} +var errWarnConflictingHint = dbterror.ClassUtil.NewStd(errno.ErrWarnConflictingHint) func init() { supportedHintNameForInsertStmt["memory_quota"] = struct{}{} @@ -118,8 +119,7 @@ func checkInsertStmtHintDuplicated(node ast.Node, sctx sessionctx.Context) { } if duplicatedHint != nil { hint := fmt.Sprintf("%s(`%v`)", duplicatedHint.HintName.O, duplicatedHint.HintData) - err := dbterror.ClassUtil.NewStd(errno.ErrWarnConflictingHint).FastGenByArgs(hint) - sctx.GetSessionVars().StmtCtx.AppendWarning(err) + sctx.GetSessionVars().StmtCtx.AppendWarning(errWarnConflictingHint.FastGenByArgs(hint)) } } } diff --git a/util/localpool/localpool_test.go b/util/localpool/localpool_test.go index 5fccedb2e5f8e..52209fabf78f7 100644 --- a/util/localpool/localpool_test.go +++ b/util/localpool/localpool_test.go @@ -20,9 +20,9 @@ package localpool import ( "math/rand" "runtime" - "sync" "testing" + "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" ) @@ -33,19 +33,19 @@ type Obj struct { func TestPool(t *testing.T) { numWorkers := runtime.GOMAXPROCS(0) - wg := new(sync.WaitGroup) - wg.Add(numWorkers) + wg := new(util.WaitGroupWrapper) pool := NewLocalPool(16, func() interface{} { return new(Obj) }, nil) n := 1000 for i := 0; i < numWorkers; i++ { - go func() { + wg.Run(func() { for j := 0; j < n; j++ { - GetAndPut(pool) + obj := pool.Get().(*Obj) + obj.val = rand.Int63() + pool.Put(obj) } - wg.Done() - }() + }) } wg.Wait() var getHit, getMiss, putHit, putMiss int diff --git a/util/mock/context.go b/util/mock/context.go index ea75700b56802..7bece0df638d9 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -38,8 +38,10 @@ import ( "github.com/tikv/client-go/v2/tikv" ) -var _ sessionctx.Context = (*Context)(nil) -var _ sqlexec.SQLExecutor = (*Context)(nil) +var ( + _ sessionctx.Context = (*Context)(nil) + _ sqlexec.SQLExecutor = (*Context)(nil) +) // Context represents mocked sessionctx.Context. type Context struct { @@ -245,7 +247,7 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return nil } if c.Store != nil { - txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(kv.GlobalTxnScope).SetStartTS(startTS)) + txn, err := c.Store.Begin(tikv.WithTxnScope(kv.GlobalTxnScope), tikv.WithStartTS(startTS)) if err != nil { return errors.Trace(err) } diff --git a/util/mock/store.go b/util/mock/store.go index 1037a763591fb..9d0ba2815cda5 100644 --- a/util/mock/store.go +++ b/util/mock/store.go @@ -38,12 +38,7 @@ func (s *Store) GetMPPClient() kv.MPPClient { return nil } func (s *Store) GetOracle() oracle.Oracle { return nil } // Begin implements kv.Storage interface. -func (s *Store) Begin() (kv.Transaction, error) { return nil, nil } - -// BeginWithOption implements kv.Storage interface. -func (s *Store) BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) { - return s.Begin() -} +func (s *Store) Begin(opts ...tikv.TxnOption) (kv.Transaction, error) { return nil, nil } // GetSnapshot implements kv.Storage interface. func (s *Store) GetSnapshot(ver kv.Version) kv.Snapshot { return nil } diff --git a/util/ranger/checker.go b/util/ranger/checker.go index a61e31a13b24c..8431bc05abef6 100644 --- a/util/ranger/checker.go +++ b/util/ranger/checker.go @@ -46,7 +46,7 @@ func (c *conditionChecker) check(condition expression.Expression) bool { } func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction) bool { - _, collation := scalar.CharsetAndCollation(scalar.GetCtx()) + _, collation := scalar.CharsetAndCollation() switch scalar.FuncName.L { case ast.LogicOr, ast.LogicAnd: return c.check(scalar.GetArgs()[0]) && c.check(scalar.GetArgs()[1]) @@ -111,7 +111,7 @@ func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction } func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) bool { - _, collation := scalar.CharsetAndCollation(scalar.GetCtx()) + _, collation := scalar.CharsetAndCollation() if !collate.CompatibleCollate(scalar.GetArgs()[0].GetType().Collate, collation) { return false } diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index e19a777006876..9b03bb48f72db 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -103,7 +103,7 @@ func getPotentialEqOrInColOffset(expr expression.Expression, cols []*expression. if !ok { return -1 } - _, collation := expr.CharsetAndCollation(f.GetCtx()) + _, collation := expr.CharsetAndCollation() switch f.FuncName.L { case ast.LogicOr: dnfItems := expression.FlattenDNFConditions(f) diff --git a/util/ranger/points.go b/util/ranger/points.go index da6489f4d7c69..a02f77cc08909 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -627,7 +627,7 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) } func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*point { - _, collation := expr.CharsetAndCollation(expr.GetCtx()) + _, collation := expr.CharsetAndCollation() if !collate.CompatibleCollate(expr.GetArgs()[0].GetType().Collate, collation) { return getFullRange() } diff --git a/util/tracing/opt_trace.go b/util/tracing/opt_trace.go new file mode 100644 index 0000000000000..6c0e7243bcdf2 --- /dev/null +++ b/util/tracing/opt_trace.go @@ -0,0 +1,81 @@ +// 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 tracing + +// LogicalPlanTrace indicates for the LogicalPlan trace information +type LogicalPlanTrace struct { + ID int + TP string + Children []*LogicalPlanTrace + + // ExplainInfo should be implemented by each implemented LogicalPlan + ExplainInfo string +} + +// LogicalOptimizeTracer indicates the trace for the whole logicalOptimize processing +type LogicalOptimizeTracer struct { + Steps []*LogicalRuleOptimizeTracer + // curRuleTracer indicates the current rule Tracer during optimize by rule + curRuleTracer *LogicalRuleOptimizeTracer +} + +// AppendRuleTracerBeforeRuleOptimize add plan tracer before optimize +func (tracer *LogicalOptimizeTracer) AppendRuleTracerBeforeRuleOptimize(name string, before *LogicalPlanTrace) { + ruleTracer := buildLogicalRuleOptimizeTracerBeforeOptimize(name, before) + tracer.Steps = append(tracer.Steps, ruleTracer) + tracer.curRuleTracer = ruleTracer +} + +// AppendRuleTracerStepToCurrent add rule optimize step to current +func (tracer *LogicalOptimizeTracer) AppendRuleTracerStepToCurrent(id int, tp, reason, action string) { + tracer.curRuleTracer.Steps = append(tracer.curRuleTracer.Steps, LogicalRuleOptimizeTraceStep{ + ID: id, + TP: tp, + Reason: reason, + Action: action, + }) +} + +// TrackLogicalPlanAfterRuleOptimize add plan trace after optimize +func (tracer *LogicalOptimizeTracer) TrackLogicalPlanAfterRuleOptimize(after *LogicalPlanTrace) { + tracer.curRuleTracer.After = after +} + +// LogicalRuleOptimizeTracer indicates the trace for the LogicalPlan tree before and after +// logical rule optimize +type LogicalRuleOptimizeTracer struct { + Before *LogicalPlanTrace + After *LogicalPlanTrace + RuleName string + Steps []LogicalRuleOptimizeTraceStep +} + +// buildLogicalRuleOptimizeTracerBeforeOptimize build rule tracer before rule optimize +func buildLogicalRuleOptimizeTracerBeforeOptimize(name string, before *LogicalPlanTrace) *LogicalRuleOptimizeTracer { + return &LogicalRuleOptimizeTracer{ + Before: before, + RuleName: name, + Steps: make([]LogicalRuleOptimizeTraceStep, 0), + } +} + +// LogicalRuleOptimizeTraceStep indicates the trace for the detailed optimize changing in +// logical rule optimize +type LogicalRuleOptimizeTraceStep struct { + Action string + Reason string + ID int + TP string +} diff --git a/util/wait_group_wrapper.go b/util/wait_group_wrapper.go new file mode 100644 index 0000000000000..16c8704920a28 --- /dev/null +++ b/util/wait_group_wrapper.go @@ -0,0 +1,53 @@ +// 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 util + +import ( + "sync" +) + +// WaitGroupWrapper is a wrapper for sync.WaitGroup +type WaitGroupWrapper struct { + sync.WaitGroup +} + +// Run runs a function in a goroutine, adds 1 to WaitGroup +// and calls done when function returns. Please DO NOT use panic +// in the cb function. +func (w *WaitGroupWrapper) Run(exec func()) { + w.Add(1) + go func() { + defer w.Done() + exec() + }() +} + +// RunWithRecover wraps goroutine startup call with force recovery, add 1 to WaitGroup +// and call done when function return. it will dump current goroutine stack into log if catch any recover result. +// exec is that execute logic function. recoverFn is that handler will be called after recover and before dump stack, +// passing `nil` means noop. +func (w *WaitGroupWrapper) RunWithRecover(exec func(), recoverFn func(r interface{})) { + w.Add(1) + go func() { + defer func() { + r := recover() + if r != nil && recoverFn != nil { + recoverFn(r) + } + w.Done() + }() + exec() + }() +} diff --git a/util/wait_group_wrapper_test.go b/util/wait_group_wrapper_test.go new file mode 100644 index 0000000000000..ea915fba64813 --- /dev/null +++ b/util/wait_group_wrapper_test.go @@ -0,0 +1,52 @@ +// 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 util + +import ( + "testing" + + "github.com/stretchr/testify/require" + "go.uber.org/atomic" +) + +func TestWaitGroupWrapperRun(t *testing.T) { + t.Parallel() + var expect int32 = 4 + var val atomic.Int32 + var wg WaitGroupWrapper + for i := int32(0); i < expect; i++ { + wg.Run(func() { + val.Inc() + }) + } + wg.Wait() + require.Equal(t, expect, val.Load()) +} + +func TestWaitGroupWrapperRunWithRecover(t *testing.T) { + t.Parallel() + var expect int32 = 2 + var val atomic.Int32 + var wg WaitGroupWrapper + for i := int32(0); i < expect; i++ { + wg.RunWithRecover(func() { + panic("test1") + }, func(r interface{}) { + val.Inc() + }) + } + wg.Wait() + require.Equal(t, expect, val.Load()) +}